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/06 17:24:51 UTC

[01/39] hive git commit: HIVE-13133: Create initial InputFormat + record readers/writers (Gunther Hagleitner)

Repository: hive
Updated Branches:
  refs/heads/master 3f07bfcef -> f089f2e64


HIVE-13133: Create initial InputFormat + record readers/writers (Gunther Hagleitner)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/bf834079
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bf834079
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bf834079

Branch: refs/heads/master
Commit: bf834079a3491bdcc65e1b839591f9db7098cf3b
Parents: bc8de94
Author: Gunther Hagleitner <gu...@apache.org>
Authored: Tue Feb 23 17:45:10 2016 -0800
Committer: Gunther Hagleitner <gu...@apache.org>
Committed: Tue Feb 23 18:53:46 2016 -0800

----------------------------------------------------------------------
 bin/ext/llapdump.sh                             |  31 +++
 bin/hive                                        |   4 +
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   2 +
 .../test/resources/testconfiguration.properties |   1 +
 jdbc/pom.xml                                    |  11 +-
 .../src/java/org/apache/hive/jdbc/LlapDump.java | 136 ++++++++++
 .../org/apache/hive/jdbc/LlapInputFormat.java   | 174 ++++++++++++
 .../hadoop/hive/llap/LlapDataOutputBuffer.java  | 165 ++++++++++++
 .../hadoop/hive/llap/LlapInputFormat.java       | 249 +++++++++++++++++
 .../apache/hadoop/hive/llap/LlapInputSplit.java | 143 ++++++++++
 .../hadoop/hive/llap/LlapOutputFormat.java      |  60 +++++
 .../hive/llap/LlapOutputFormatService.java      | 141 ++++++++++
 .../hadoop/hive/llap/LlapRecordReader.java      |  86 ++++++
 .../hadoop/hive/llap/LlapRecordWriter.java      |  52 ++++
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   2 +
 .../hive/ql/exec/SerializationUtilities.java    |   2 +-
 .../hive/ql/exec/tez/HiveSplitGenerator.java    |  21 +-
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |   2 +-
 .../hive/ql/optimizer/SimpleFetchOptimizer.java |  50 ++--
 .../ql/udf/generic/GenericUDFGetSplits.java     | 265 +++++++++++++++++++
 .../hadoop/hive/llap/TestLlapOutputFormat.java  | 124 +++++++++
 .../queries/clientpositive/udf_get_splits.q     |   6 +
 .../clientpositive/tez/udf_get_splits.q.out     |  73 +++++
 23 files changed, 1758 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/bin/ext/llapdump.sh
----------------------------------------------------------------------
diff --git a/bin/ext/llapdump.sh b/bin/ext/llapdump.sh
new file mode 100644
index 0000000..2564e82
--- /dev/null
+++ b/bin/ext/llapdump.sh
@@ -0,0 +1,31 @@
+# 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.
+
+THISSERVICE=llapdump
+export SERVICE_LIST="${SERVICE_LIST}${THISSERVICE} "
+
+llapdump () {
+  CLASS=org.apache.hive.jdbc.LlapDump
+  HIVE_OPTS=''
+  execHiveCmd $CLASS "$@"
+}
+
+llapdump_help () {
+  echo "usage ./hive llapdump [-l <url>] [-u <user>] [-p <pwd>] <query>"
+  echo ""
+  echo "  --location (-l)  hs2 url"
+  echo "  --user (-u)      user name"
+  echo "  --pwd (-p)       password"
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/bin/hive
----------------------------------------------------------------------
diff --git a/bin/hive b/bin/hive
index e9477f7..e6693f6 100755
--- a/bin/hive
+++ b/bin/hive
@@ -48,6 +48,10 @@ while [ $# -gt 0 ]; do
       SERVICE=orcfiledump
       shift
       ;;
+    --llapdump)
+      SERVICE=llapdump
+      shift
+      ;;
     --help)
       HELP=_help
       shift

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 9cb626e..7fbcbba 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2684,6 +2684,8 @@ public class HiveConf extends Configuration {
         false,
         "Whether to setup split locations to match nodes on which llap daemons are running," +
             " instead of using the locations provided by the split itself"),
+    LLAP_DAEMON_OUTPUT_SERVICE_PORT("hive.llap.daemon.output.service.port", 15003,
+        "LLAP daemon output service port"),
 
     SPARK_CLIENT_FUTURE_TIMEOUT("hive.spark.client.future.timeout",
       "60s", new TimeValidator(TimeUnit.SECONDS),

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 58d0a45..deb9905 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -435,6 +435,7 @@ minitez.query.files=bucket_map_join_tez1.q,\
   tez_smb_main.q,\
   tez_smb_1.q,\
   tez_smb_empty.q,\
+  udf_get_splits.q,\
   vector_join_part_col_char.q,\
   vectorized_dynamic_partition_pruning.q,\
   tez_multi_union.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index f87ab59..2be8c30 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -42,14 +42,13 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
+      <artifactId>hive-exec</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
       <artifactId>hive-service</artifactId>
       <version>${project.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.hive</groupId>
-            <artifactId>hive-exec</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java b/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
new file mode 100644
index 0000000..b0c0253
--- /dev/null
+++ b/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.jdbc;
+
+import java.io.OutputStream;
+import java.io.InputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.FileInputStream;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.io.RCFile.Reader;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.RecordWriter;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.llap.io.api.LlapProxy;
+import org.apache.hadoop.hive.llap.LlapRecordReader;
+import org.apache.hadoop.hive.metastore.api.Schema;
+
+public class LlapDump {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LlapDump.class);
+
+  private static String url = "jdbc:hive2://localhost:10000/default";
+  private static String user = "hive";
+  private static String pwd = "";
+  private static String query = "select * from test";
+
+  public static void main(String[] args) throws Exception {
+    Options opts = createOptions();
+    CommandLine cli = new GnuParser().parse(opts, args);
+
+    if (cli.hasOption('h')) {
+      HelpFormatter formatter = new HelpFormatter();
+      formatter.printHelp("orcfiledump", opts);
+      return;
+    }
+
+    if (cli.hasOption('l')) {
+      url = cli.getOptionValue("l");
+    }
+
+    if (cli.hasOption('u')) {
+      user = cli.getOptionValue("u");
+    }
+
+    if (cli.hasOption('p')) {
+      pwd = cli.getOptionValue("p");
+    }
+
+    if (cli.getArgs().length > 0) {
+      query = cli.getArgs()[0];
+    }
+
+    System.out.println("url: "+url);
+    System.out.println("user: "+user);
+    System.out.println("query: "+query);
+
+    LlapInputFormat format = new LlapInputFormat(url, user, pwd, query);
+    JobConf job = new JobConf();
+    InputSplit[] splits = format.getSplits(job, 1);
+    RecordReader<NullWritable, Text> reader = format.getRecordReader(splits[0], job, null);
+
+    if (reader instanceof LlapRecordReader) {
+      Schema schema = ((LlapRecordReader)reader).getSchema();
+      System.out.println(""+schema);
+    }
+    System.out.println("Results: ");
+    System.out.println("");
+
+    Text value = reader.createValue();
+    while (reader.next(NullWritable.get(), value)) {
+      System.out.println(value);
+    }
+  }
+
+  static Options createOptions() {
+    Options result = new Options();
+
+    result.addOption(OptionBuilder
+        .withLongOpt("location")
+        .withDescription("HS2 url")
+	.hasArg()
+        .create('l'));
+
+    result.addOption(OptionBuilder
+        .withLongOpt("user")
+        .withDescription("user name")
+        .hasArg()
+        .create('u'));
+
+    result.addOption(OptionBuilder
+        .withLongOpt("pwd")
+        .withDescription("password")
+        .hasArg()
+        .create('p'));
+
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java b/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
new file mode 100644
index 0000000..97fe2c5
--- /dev/null
+++ b/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
@@ -0,0 +1,174 @@
+/*
+ * 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.jdbc;
+
+import java.util.ArrayList;
+import java.util.List;
+
+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 org.apache.hadoop.fs.FileSystem;
+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.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.Progressable;
+
+import com.google.common.base.Preconditions;
+
+public class LlapInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
+
+  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;
+
+  private Connection con;
+  private Statement stmt;
+
+  public LlapInputFormat(String url, String user, String pwd, String query) {
+    this.url = url;
+    this.user = user;
+    this.pwd = pwd;
+    this.query = query;
+  }
+
+  public LlapInputFormat() {}
+
+  public class LlapInputSplit implements InputSplitWithLocationInfo {
+    InputSplitWithLocationInfo nativeSplit;
+    String inputFormatClassName;
+
+    @Override
+    public long getLength() throws IOException {
+      return nativeSplit.getLength();
+    }
+
+    @Override
+    public String[] getLocations() throws IOException {
+      return nativeSplit.getLocations();
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      out.writeUTF(inputFormatClassName);
+      out.writeUTF(nativeSplit.getClass().toString());
+      nativeSplit.write(out);
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      inputFormatClassName = in.readUTF();
+      String splitClass = in.readUTF();
+      try {
+        nativeSplit = (InputSplitWithLocationInfo)Class.forName(splitClass).newInstance();
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+      nativeSplit.readFields(in);
+    }
+
+    @Override
+    public SplitLocationInfo[] getLocationInfo() throws IOException {
+      return nativeSplit.getLocationInfo();
+    }
+
+    public InputSplit getSplit() {
+      return nativeSplit;
+    }
+
+    public InputFormat<NullWritable, V> getInputFormat() {
+      try {
+        return (InputFormat<NullWritable, V>) Class.forName(inputFormatClassName)
+            .newInstance();
+      } catch(Exception e) {
+        return null;
+      }
+    }
+  }
+
+  @Override
+  public RecordReader<NullWritable, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    LlapInputSplit llapSplit = (LlapInputSplit)split;
+    return llapSplit.getInputFormat().getRecordReader(llapSplit.getSplit(), job, reporter);
+  }
+
+  @Override
+  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
+    List<InputSplit> ins = new ArrayList<InputSplit>();
+
+    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 = "select r.if_class as ic, r.split_class as sc, r.split as s from (select explode(get_splits(\""+query+"\","+numSplits+")) as r) t";
+      ResultSet res = stmt.executeQuery(sql);
+      while (res.next()) {
+        // deserialize split
+        DataInput in = new DataInputStream(new ByteArrayInputStream(res.getBytes(3)));
+        InputSplit is = (InputSplitWithLocationInfo)Class.forName(res.getString(2)).newInstance(); // todo setAccessible on ctor
+        is.readFields(in);
+        ins.add(is);
+      }
+
+      res.close();
+      stmt.close();
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+    return ins.toArray(new InputSplit[ins.size()]); // todo wrap input split with format
+  }
+
+  public void close() {
+    try {
+      con.close();
+    } catch (Exception e) {
+      // ignore
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/java/org/apache/hadoop/hive/llap/LlapDataOutputBuffer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapDataOutputBuffer.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapDataOutputBuffer.java
new file mode 100644
index 0000000..aad8968
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapDataOutputBuffer.java
@@ -0,0 +1,165 @@
+/**
+ * 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.ql.llap;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+/**
+ * A thread-not-safe version of Hadoop's DataOutputBuffer, which removes all
+ * synchronized modifiers.
+ */
+public class LlapDataOutputBuffer implements DataOutput {
+
+  int readOffset;
+  int writeOffset;
+  byte[] buffer;
+
+  /** Constructs a new empty buffer. */
+  public LlapDataOutputBuffer(int length) {
+    buffer = new byte[length];
+    reset();
+  }
+
+  /**
+   * Returns the current contents of the buffer. Data is only valid to
+   * {@link #getLength()}.
+   */
+  public byte[] getData() {
+    return buffer;
+  }
+
+  /** Returns the length of the valid data currently in the buffer. */
+  public int getLength() {
+    return (writeOffset - readOffset) % buffer.length;
+  }
+
+  /** Resets the buffer to empty. */
+  public LlapDataOutputBuffer reset() {
+    readOffset = 0;
+    writeOffset = 0;
+    return this;
+  }
+
+  /** Writes bytes from a DataInput directly into the buffer. */
+  public void write(DataInput in, int length) throws IOException {
+    //
+  }
+
+  @Override
+  public synchronized void write(int b) throws IOException {
+    while (readOffset == writeOffset) {
+      try {
+	wait();
+      } catch(InterruptedException e) {
+      }
+    }
+    buffer[writeOffset] = (byte)b;
+    writeOffset = (writeOffset + 1) % buffer.length;
+    notify();
+  }
+
+  public synchronized int read() throws IOException {
+    while (readOffset == writeOffset) {
+      try {
+	wait();
+      } catch(InterruptedException e) {
+      }
+    }
+    int b = buffer[readOffset];
+    readOffset = (readOffset + 1) % buffer.length;
+    notify();
+    return b;
+  }
+
+  @Override
+  public void write(byte b[], int off, int len) throws IOException {
+    while(len-- != 0) {
+      write(b[off++]);
+    }
+  }
+
+  @Override
+  public void write(byte b[]) throws IOException {
+    write(b, 0, b.length);
+  }
+
+
+  @Override
+  public void writeBoolean(boolean v) throws IOException {
+    write(v?1:0);
+  }
+
+  @Override
+  public void writeByte(int v) throws IOException  {
+    write(v);
+  }
+
+  @Override
+  public void writeChar(int v) throws IOException  {
+    write(v);
+  }
+
+  @Override
+  public void writeBytes(String v) throws IOException  {
+    write(v.getBytes(), 0, v.length());
+  }
+
+  @Override
+  public void writeChars(String v) throws IOException  {
+    write(v.getBytes(), 0, v.length());
+  }
+
+  @Override
+  public void writeDouble(double v) throws IOException  {
+    write(ByteBuffer.allocate(8).putDouble(v).array(),0,8);
+  }
+
+  @Override
+  public void writeFloat(float v) throws IOException  {
+    write(ByteBuffer.allocate(4).putFloat(v).array(),0,4);
+  }
+
+  @Override
+  public void writeInt(int v) throws IOException  {
+    write(v);
+    write(v>>>8);
+    write(v>>>16);
+    write(v>>>24);
+  }
+
+  @Override
+  public void writeLong(long v) throws IOException  {
+    int v1 = (int)v;
+    int v2 = (int)v>>>32;
+    write(v1);
+    write(v2);
+  }
+
+  @Override
+  public void writeShort(int v) throws IOException  {
+    write(v);
+    write(v>>>8);
+  }
+
+  @Override
+  public void writeUTF(String v) throws IOException  {
+    write(v.getBytes(), 0, v.length());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
new file mode 100644
index 0000000..4db4d32
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
@@ -0,0 +1,249 @@
+/*
+ * 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 javax.security.auth.login.LoginException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.Socket;
+
+import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
+
+import com.esotericsoftware.kryo.Kryo;
+import java.io.ByteArrayOutputStream;
+import java.io.OutputStream;
+import java.io.InputStream;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URISyntaxException;
+import java.io.FileNotFoundException;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.SplitLocationInfo;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.runtime.api.Event;
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.exec.tez.DagUtils;
+import org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator;
+import org.apache.hadoop.hive.ql.plan.MapWork;
+import org.apache.hadoop.hive.ql.plan.TezWork;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.tez.dag.api.TaskLocationHint;
+import org.apache.tez.dag.api.VertexLocationHint;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.mapreduce.hadoop.InputSplitInfoMem;
+import org.apache.tez.mapreduce.hadoop.MRInputHelpers;
+import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRInputUserPayloadProto;
+import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitProto;
+import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitsProto;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.InputInitializer;
+import org.apache.tez.runtime.api.InputInitializerContext;
+import org.apache.tez.runtime.api.InputSpecUpdate;
+import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
+import org.apache.tez.runtime.api.events.InputInitializerEvent;
+
+
+import com.google.common.base.Preconditions;
+
+public class LlapInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LlapInputFormat.class);
+
+  private TezWork work;
+  private Schema schema;
+
+  public LlapInputFormat(TezWork tezWork, Schema schema) {
+    this.work = tezWork;
+    this.schema = schema;
+  }
+
+  // need empty constructor for bean instantiation
+  public LlapInputFormat() {}
+
+  /*
+   * This proxy record reader has the duty of establishing a connected socket with LLAP, then fire
+   * off the work in the split to LLAP and finally return the connected socket back in an
+   * LlapRecordReader. The LlapRecordReader class reads the results from the socket.
+   */
+  public RecordReader<NullWritable, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+
+    LlapInputSplit llapSplit = (LlapInputSplit)split;
+
+    // TODO: push event into LLAP
+
+    // this is just the portion that sets up the io to receive data
+    String host = split.getLocations()[0];
+    String id = job.get(LlapOutputFormat.LLAP_OF_ID_KEY);
+
+    HiveConf conf = new HiveConf();
+    Socket socket = new Socket(host,
+        conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT));
+
+    LOG.debug("Socket connected");
+
+    socket.getOutputStream().write(id.getBytes());
+    socket.getOutputStream().write(0);
+    socket.getOutputStream().flush();
+
+    LOG.debug("Registered id: " + id);
+
+    return new LlapRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class);
+  }
+
+  /*
+   * getSplits() gets called as part of the GenericUDFGetSplits call to get splits. Here we create
+   * an array of input splits from the work item we have, figure out the location for llap and pass
+   * that back for the submission. getRecordReader method above uses that split info to assign the
+   * work to llap.
+   */
+  @Override
+  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
+    // TODO: need to build proto of plan
+
+    DAG dag = DAG.create(work.getName());
+    dag.setCredentials(job.getCredentials());
+    // TODO: set access control? TezTask.setAccessControlsForCurrentUser(dag);
+
+    DagUtils utils = DagUtils.getInstance();
+    Context ctx = new Context(job);
+    MapWork mapWork = (MapWork) work.getAllWork().get(0);
+    // bunch of things get setup in the context based on conf but we need only the MR tmp directory
+    // for the following method.
+    JobConf wxConf = utils.initializeVertexConf(job, ctx, mapWork);
+    Path scratchDir = utils.createTezDir(ctx.getMRScratchDir(), job);
+    FileSystem fs = scratchDir.getFileSystem(job);
+    try {
+      LocalResource appJarLr = createJarLocalResource(utils.getExecJarPathLocal(), utils, job);
+      Vertex wx = utils.createVertex(wxConf, mapWork, scratchDir, appJarLr,
+          new ArrayList<LocalResource>(), fs, ctx, false, work,
+          work.getVertexType(mapWork));
+      dag.addVertex(wx);
+      utils.addCredentials(mapWork, dag);
+
+      // we have the dag now proceed to get the splits:
+      HiveSplitGenerator splitGenerator = new HiveSplitGenerator(null);
+      splitGenerator.initializeSplitGenerator(wxConf, mapWork);
+      List<Event> eventList = splitGenerator.initialize();
+
+      // hack - just serializing with kryo for now. This needs to be done properly
+      InputSplit[] result = new InputSplit[eventList.size()];
+      int i = 0;
+      ByteArrayOutputStream bos = new ByteArrayOutputStream(10240);
+
+      InputConfigureVertexTasksEvent configureEvent = (InputConfigureVertexTasksEvent)
+	eventList.remove(0);
+
+      List<TaskLocationHint> hints = configureEvent.getLocationHint().getTaskLocationHints();
+      for (Event event: eventList) {
+	TaskLocationHint hint = hints.remove(0);
+        Set<String> hosts = hint.getHosts();
+	SplitLocationInfo[] locations = new SplitLocationInfo[hosts.size()];
+
+	int j = 0;
+	for (String host: hosts) {
+	  locations[j++] = new SplitLocationInfo(host,false);
+	}
+
+	bos.reset();
+	Kryo kryo = SerializationUtilities.borrowKryo();
+	SerializationUtilities.serializeObjectByKryo(kryo, event, bos);
+	SerializationUtilities.releaseKryo(kryo);
+	result[i++] = new LlapInputSplit(bos.toByteArray(), locations, schema);
+      }
+      return result;
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Returns a local resource representing a jar. This resource will be used to execute the plan on
+   * the cluster.
+   *
+   * @param localJarPath
+   *          Local path to the jar to be localized.
+   * @return LocalResource corresponding to the localized hive exec resource.
+   * @throws IOException
+   *           when any file system related call fails.
+   * @throws LoginException
+   *           when we are unable to determine the user.
+   * @throws URISyntaxException
+   *           when current jar location cannot be determined.
+   */
+  private LocalResource createJarLocalResource(String localJarPath, DagUtils utils,
+      Configuration conf)
+    throws IOException, LoginException, IllegalArgumentException, FileNotFoundException {
+    FileStatus destDirStatus = utils.getHiveJarDirectory(conf);
+    assert destDirStatus != null;
+    Path destDirPath = destDirStatus.getPath();
+
+    Path localFile = new Path(localJarPath);
+    String sha = getSha(localFile, conf);
+
+    String destFileName = localFile.getName();
+
+    // Now, try to find the file based on SHA and name. Currently we require exact name match.
+    // We could also allow cutting off versions and other stuff provided that SHA matches...
+    destFileName = FilenameUtils.removeExtension(destFileName) + "-" + sha
+      + FilenameUtils.EXTENSION_SEPARATOR + FilenameUtils.getExtension(destFileName);
+
+    // TODO: if this method is ever called on more than one jar, getting the dir and the
+    // list need to be refactored out to be done only once.
+    Path destFile = new Path(destDirPath.toString() + "/" + destFileName);
+    return utils.localizeResource(localFile, destFile, LocalResourceType.FILE, conf);
+  }
+
+  private String getSha(Path localFile, Configuration conf)
+    throws IOException, IllegalArgumentException {
+    InputStream is = null;
+    try {
+      FileSystem localFs = FileSystem.getLocal(conf);
+      is = localFs.open(localFile);
+      return DigestUtils.sha256Hex(is);
+    } finally {
+      if (is != null) {
+        is.close();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
new file mode 100644
index 0000000..78dbb34
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.llap;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataInputStream;
+import java.io.ByteArrayInputStream;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputFormat;
+import org.apache.hadoop.mapred.RecordWriter;
+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.hive.llap.io.api.LlapProxy;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.SplitLocationInfo;
+import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.AutoExpandingBufferWriteTransport;
+import org.apache.thrift.transport.AutoExpandingBuffer;
+
+import com.google.common.base.Preconditions;
+
+public class LlapInputSplit implements InputSplitWithLocationInfo {
+
+  byte[] queryFragment;
+  SplitLocationInfo[] locations;
+  Schema schema;
+
+  public LlapInputSplit() {}
+
+  public LlapInputSplit(byte[] queryFragment, SplitLocationInfo[] locations, Schema schema) {
+    this.queryFragment = queryFragment;
+    this.locations = locations;
+    this.schema = schema;
+  }
+
+  public Schema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public long getLength() throws IOException {
+    return 0;
+  }
+
+  @Override
+  public String[] getLocations() throws IOException {
+    String[] locs = new String[locations.length];
+    for (int i = 0; i < locations.length; ++i) {
+      locs[i] = locations[i].getLocation();
+    }
+    return locs;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(queryFragment.length);
+    out.write(queryFragment);
+
+    out.writeInt(locations.length);
+    for (int i = 0; i < locations.length; ++i) {
+      out.writeUTF(locations[i].getLocation());
+    }
+
+    byte[] binarySchema;
+
+    try {
+      AutoExpandingBufferWriteTransport transport = new AutoExpandingBufferWriteTransport(1024, 2d);
+      TProtocol protocol = new TBinaryProtocol(transport);
+      schema.write(protocol);
+      binarySchema = transport.getBuf().array();
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+
+    out.writeInt(binarySchema.length);
+    out.write(binarySchema);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    byte[] queryFragment;
+
+    int length = in.readInt();
+    queryFragment = new byte[length];
+    in.readFully(queryFragment);
+
+    length = in.readInt();
+    locations = new SplitLocationInfo[length];
+
+    for (int i = 0; i < length; ++i) {
+      locations[i] = new SplitLocationInfo(in.readUTF(), false);
+    }
+
+    length = in.readInt();
+
+    try {
+      AutoExpandingBufferWriteTransport transport = new AutoExpandingBufferWriteTransport(length, 2d);
+      AutoExpandingBuffer buf = transport.getBuf();
+      in.readFully(buf.array(), 0, length);
+
+      TProtocol protocol = new TBinaryProtocol(transport);
+      schema = new Schema();
+      schema.read(protocol);
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public SplitLocationInfo[] getLocationInfo() throws IOException {
+    return locations;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormat.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormat.java
new file mode 100644
index 0000000..8e98aba
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormat.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.llap;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputFormat;
+import org.apache.hadoop.mapred.RecordWriter;
+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.hive.llap.io.api.LlapProxy;
+
+import com.google.common.base.Preconditions;
+
+/**
+ *
+ */
+public class LlapOutputFormat<K extends Writable, V extends Writable>
+  implements OutputFormat<K, V> {
+
+  public static final String LLAP_OF_ID_KEY = "llap.of.id";
+
+  @Override
+  public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException {
+  }
+
+  @Override
+  public RecordWriter<K, V> getRecordWriter(FileSystem ignored, JobConf job, String name, Progressable progress) throws IOException {
+    if (!LlapProxy.isDaemon()) {
+      throw new IOException("LlapOutputFormat can only be used inside Llap");
+    }
+    try {
+      return LlapOutputFormatService.get().<K,V>getWriter(job.get(LLAP_OF_ID_KEY));
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
new file mode 100644
index 0000000..4f38ff1
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
@@ -0,0 +1,141 @@
+/*
+ * 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.Map;
+import java.util.HashMap;
+import java.io.IOException;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputFormat;
+import org.apache.hadoop.mapred.RecordWriter;
+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.hive.conf.HiveConf;
+import org.apache.hadoop.hive.llap.io.api.LlapProxy;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ *
+ */
+public class LlapOutputFormatService {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LlapOutputFormat.class);
+
+  private static LlapOutputFormatService service;
+  private final Map<String, RecordWriter> writers;
+  private final ServerSocket socket;
+  private final HiveConf conf;
+  private final ExecutorService executor;
+  private static final int WAIT_TIME = 5;
+
+  private LlapOutputFormatService() throws IOException {
+    writers = new HashMap<String, RecordWriter>();
+    conf = new HiveConf();
+    executor = Executors.newSingleThreadExecutor(
+      new ThreadFactoryBuilder().setDaemon(true).setNameFormat("LLAP output %d").build());
+    socket = new ServerSocket(
+      conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT));
+  }
+
+  public static LlapOutputFormatService get() throws IOException {
+    if (service == null) {
+      service = new LlapOutputFormatService();
+      service.start();
+    }
+    return service;
+  }
+
+  public void start() throws IOException {
+    executor.submit(new Runnable() {
+      byte[] buffer = new byte[4096];
+      @Override
+      public void run() {
+	while (true) {
+	  Socket s = null;
+	  try {
+	    s = socket.accept();
+	    String id = readId(s);
+	    LOG.debug("Received: "+id);
+	    registerReader(s, id);
+	  } catch (IOException io) {
+	    if (s != null) {
+	      try{
+		s.close();
+	      } catch (IOException io2) {
+		// ignore
+	      }
+	    }
+	  }
+	}
+      }
+
+    private String readId(Socket s) throws IOException {
+      InputStream in = s.getInputStream();
+      int idx = 0;
+      while((buffer[idx++] = (byte)in.read()) != '\0') {}
+      return new String(buffer,0,idx-1);
+    }
+
+    private void registerReader(Socket s, String id) throws IOException {
+      synchronized(service) {
+	LOG.debug("registering socket for: "+id);
+	LlapRecordWriter writer = new LlapRecordWriter(s.getOutputStream());
+        writers.put(id, writer);
+        service.notifyAll();
+      }
+    }
+    }
+    );
+  }
+
+  public void stop() throws IOException, InterruptedException {
+    executor.shutdown();
+    executor.awaitTermination(WAIT_TIME, TimeUnit.SECONDS);
+    socket.close();
+  }
+
+  public <K,V> RecordWriter<K, V> getWriter(String id) throws IOException, InterruptedException {
+    RecordWriter writer = null;
+    synchronized(service) {
+      while ((writer = writers.get(id)) == null) {
+	LOG.debug("Waiting for writer for: "+id);
+	service.wait();
+      }
+    }
+    return writer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
new file mode 100644
index 0000000..ce3d39a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.llap;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.DataInputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.io.RCFile.Reader;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.hive.metastore.api.Schema;
+
+public class LlapRecordReader<V extends WritableComparable> implements RecordReader<NullWritable, V> {
+
+  DataInputStream din;
+  Schema schema;
+  Class<V> clazz;
+
+  public LlapRecordReader(InputStream in, Schema schema, Class<V> clazz) {
+    din = new DataInputStream(in);
+    this.schema = schema;
+    this.clazz = clazz;
+  }
+
+  public Schema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public void close() throws IOException {
+    din.close();
+  }
+
+  @Override
+  public long getPos() { return 0; }
+
+  @Override
+  public float getProgress() { return 0f; }
+
+  @Override
+  public NullWritable createKey() {
+    return NullWritable.get();
+  }
+
+  @Override
+  public V createValue() {
+    try {
+      return clazz.newInstance();
+    } catch (Exception e) {
+      return null;
+    }
+  }
+
+  @Override
+  public boolean next(NullWritable key, V value) {
+    try {
+      value.readFields(din);
+      return true;
+    } catch (IOException io) {
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordWriter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordWriter.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordWriter.java
new file mode 100644
index 0000000..4d1996c
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordWriter.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.llap;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.DataOutputStream;;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.mapred.RecordWriter;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.JobConf;
+
+public class LlapRecordWriter<K extends Writable, V extends WritableComparable>
+  implements RecordWriter<K,V> {
+
+  DataOutputStream dos;
+
+  public LlapRecordWriter(OutputStream out) {
+    dos = new DataOutputStream(out);
+  }
+
+  @Override
+  public void close(Reporter reporter) throws IOException {
+    dos.close();
+  }
+
+  @Override
+  public void write(K key, V value) throws IOException {
+    value.write(dos);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index e0e030f..f3afa24 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -344,6 +344,8 @@ public final class FunctionRegistry {
     system.registerGenericUDF("ewah_bitmap_or", GenericUDFEWAHBitmapOr.class);
     system.registerGenericUDF("ewah_bitmap_empty", GenericUDFEWAHBitmapEmpty.class);
 
+    system.registerGenericUDF("get_splits", GenericUDFGetSplits.class);
+
     // Aliases for Java Class Names
     // These are used in getImplicitConvertUDFMethod
     system.registerUDF(serdeConstants.BOOLEAN_TYPE_NAME, UDFToBoolean.class, false, UDFToBoolean.class.getSimpleName());

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
index b05a79e..eaa4293 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
@@ -570,7 +570,7 @@ public class SerializationUtilities {
    * @param plan Usually of type MapredWork, MapredLocalWork etc.
    * @param out stream in which serialized plan is written into
    */
-  private static void serializeObjectByKryo(Kryo kryo, Object plan, OutputStream out) {
+  public static void serializeObjectByKryo(Kryo kryo, Object plan, OutputStream out) {
     Output output = new Output(out);
     kryo.setClassLoader(Utilities.getSessionSpecifiedClassLoader());
     kryo.writeObject(output, plan);

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
index 8e48c2e..b0cda82 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
@@ -74,17 +74,26 @@ public class HiveSplitGenerator extends InputInitializer {
 
   private static final Logger LOG = LoggerFactory.getLogger(HiveSplitGenerator.class);
 
-  private final DynamicPartitionPruner pruner;
-  private final Configuration conf;
-  private final JobConf jobConf;
-  private final MRInputUserPayloadProto userPayloadProto;
-  private final MapWork work;
+  private DynamicPartitionPruner pruner = null;
+  private Configuration conf = null;
+  private JobConf jobConf = null;
+  private MRInputUserPayloadProto userPayloadProto = null;
+  private MapWork work = null;
   private final SplitGrouper splitGrouper = new SplitGrouper();
-  private final SplitLocationProvider splitLocationProvider;
+  private SplitLocationProvider splitLocationProvider = null;
+
+  public void initializeSplitGenerator(Configuration conf, MapWork work) {
+    this.conf = conf;
+    this.work = work;
+    this.jobConf = new JobConf(conf);
+  }
 
   public HiveSplitGenerator(InputInitializerContext initializerContext) throws IOException,
       SerDeException {
     super(initializerContext);
+    if (initializerContext == null) {
+      return;
+    }
     Preconditions.checkNotNull(initializerContext);
     userPayloadProto =
         MRInputHelpers.parseMRInputPayload(initializerContext.getInputUserPayload());

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
index 83defea..9e688ea 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
@@ -412,7 +412,7 @@ public class TezTask extends Task<TezWork> {
     return dag;
   }
 
-  private void setAccessControlsForCurrentUser(DAG dag) {
+  public static void setAccessControlsForCurrentUser(DAG dag) {
     // get current user
     String currentUser = SessionState.getUserFromAuthenticator();
     if(LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
index b5ceb14..ca8dccf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.ql.exec.FetchTask;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.LimitOperator;
+import org.apache.hadoop.hive.ql.exec.UDTFOperator;
 import org.apache.hadoop.hive.ql.exec.ListSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorFactory;
@@ -140,35 +141,31 @@ public class SimpleFetchOptimizer extends Transform {
   }
 
   private boolean checkThreshold(FetchData data, int limit, ParseContext pctx) throws Exception {
+    boolean result = false;
+
     if (limit > 0) {
       if (data.hasOnlyPruningFilter()) {
         /* partitioned table + query has only pruning filters */
-        return true;
+        result = true;
       } else if (data.isPartitioned() == false && data.isFiltered() == false) {
         /* unpartitioned table + no filters */
-        return true;
+        result = true;
       }
       /* fall through */
-    }
-    long threshold = HiveConf.getLongVar(pctx.getConf(),
-        HiveConf.ConfVars.HIVEFETCHTASKCONVERSIONTHRESHOLD);
-    if (threshold < 0) {
-      return true;
-    }
-    Operator child = data.scanOp.getChildOperators().get(0);
-    if(child instanceof SelectOperator) {
-      // select *, constant and casts can be allowed without a threshold check
-      if (checkExpressions((SelectOperator)child)) {
-        return true;
+    } else {
+      long threshold = HiveConf.getLongVar(pctx.getConf(),
+	  HiveConf.ConfVars.HIVEFETCHTASKCONVERSIONTHRESHOLD);
+      if (threshold < 0) {
+	result = true;
+      } else {
+	long remaining = threshold;
+	remaining -= data.getInputLength(pctx, remaining);
+	if (remaining >= 0) {
+	  result = true;
+	}
       }
     }
-    long remaining = threshold;
-    remaining -= data.getInputLength(pctx, remaining);
-    if (remaining < 0) {
-      LOG.info("Threshold " + remaining + " exceeded for pseudoMR mode");
-      return false;
-    }
-    return true;
+    return result;
   }
 
   // all we can handle is LimitOperator, FilterOperator SelectOperator and final FS
@@ -187,23 +184,20 @@ public class SimpleFetchOptimizer extends Transform {
       return null;
     }
     Table table = ts.getConf().getTableMetadata();
-    if (table == null) {
-      return null;
-    }
     ReadEntity parent = PlanUtils.getParentViewInfo(alias, pctx.getViewAliasToInput());
-    if (!table.isPartitioned()) {
+    if (table != null && !table.isPartitioned()) {
       FetchData fetch = new FetchData(ts, parent, table, splitSample);
       return checkOperators(fetch, aggressive, false);
     }
 
     boolean bypassFilter = false;
-    if (HiveConf.getBoolVar(pctx.getConf(), HiveConf.ConfVars.HIVEOPTPPD)) {
+    if (table != null && HiveConf.getBoolVar(pctx.getConf(), HiveConf.ConfVars.HIVEOPTPPD)) {
       ExprNodeDesc pruner = pctx.getOpToPartPruner().get(ts);
       if (PartitionPruner.onlyContainsPartnCols(table, pruner)) {
         bypassFilter = !pctx.getPrunedPartitions(alias, ts).hasUnknownPartitions();
       }
     }
-    if (!aggressive && !bypassFilter) {
+    if (table != null && !aggressive && !bypassFilter) {
       return null;
     }
     PrunedPartitionList partitions = pctx.getPrunedPartitions(alias, ts);
@@ -231,7 +225,7 @@ public class SimpleFetchOptimizer extends Transform {
         continue;
       }
 
-      if (!(op instanceof LimitOperator || (op instanceof FilterOperator && bypassFilter))) {
+      if (!(op instanceof LimitOperator || (op instanceof FilterOperator && bypassFilter)) || op instanceof UDTFOperator) {
         break;
       }
 
@@ -289,7 +283,7 @@ public class SimpleFetchOptimizer extends Transform {
 
   private boolean isConvertible(FetchData fetch, Operator<?> operator, Set<Operator<?>> traversed) {
     if (operator instanceof ReduceSinkOperator || operator instanceof CommonJoinOperator
-        || operator instanceof ScriptOperator || operator instanceof UDTFOperator) {
+        || operator instanceof ScriptOperator) {
       return false;
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
new file mode 100644
index 0000000..3b7dcd9
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
@@ -0,0 +1,265 @@
+/**
+ * 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.ql.udf.generic;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.UUID;
+import java.io.Serializable;
+import java.io.IOException;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.DataOutput;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hive.ql.udf.UDFType;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.tez.TezTask;
+import org.apache.hadoop.hive.ql.plan.TezWork;
+import org.apache.hadoop.hive.ql.plan.MapWork;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.MapredContext;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.llap.LlapInputFormat;
+import org.apache.hadoop.hive.llap.LlapOutputFormat;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.JobConfigurable;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.metastore.api.Schema;
+
+
+/**
+ * GenericUDFGetSplits.
+ *
+ */
+@Description(name = "get_splits", value = "_FUNC_(string,int) - "
+    + "Returns an array of length int serialized splits for the referenced tables string.")
+@UDFType(deterministic = false)
+public class GenericUDFGetSplits extends GenericUDF {
+
+  private static final Logger LOG = LoggerFactory.getLogger(GenericUDFGetSplits.class);
+
+  private transient StringObjectInspector stringOI;
+  private transient IntObjectInspector intOI;
+  private final ArrayList<Object> retArray = new ArrayList<Object>();
+  private transient JobConf jc;
+  private transient Hive db;
+  private ByteArrayOutputStream bos;
+  private DataOutput dos;
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments)
+    throws UDFArgumentException {
+
+    LOG.debug("initializing GenericUDFGetSplits");
+
+    try {
+      if (SessionState.get() != null && SessionState.get().getConf() != null) {
+        HiveConf conf = SessionState.get().getConf();
+        jc = new JobConf(conf);
+        db = Hive.get(conf);
+      } else {
+        jc = MapredContext.get().getJobConf();
+        db = Hive.get();
+      }
+    } catch(HiveException e) {
+      LOG.error("Failed to initialize: ",e);
+      throw new UDFArgumentException(e);
+    }
+
+    LOG.debug("Initialized conf, jc and metastore connection");
+
+    if (arguments.length != 2) {
+      throw new UDFArgumentLengthException("The function GET_SPLITS accepts 2 arguments.");
+    } else if (!(arguments[0] instanceof StringObjectInspector)) {
+      LOG.error("Got "+arguments[0].getTypeName()+" instead of string.");
+      throw new UDFArgumentTypeException(0, "\""
+	  + "string\" is expected at function GET_SPLITS, " + "but \""
+	  + arguments[0].getTypeName() + "\" is found");
+    } else if (!(arguments[1] instanceof IntObjectInspector)) {
+      LOG.error("Got "+arguments[1].getTypeName()+" instead of int.");
+      throw new UDFArgumentTypeException(1, "\""
+	  + "int\" is expected at function GET_SPLITS, " + "but \""
+	  + arguments[1].getTypeName() + "\" is found");
+    }
+
+    stringOI = (StringObjectInspector) arguments[0];
+    intOI = (IntObjectInspector) arguments[1];
+
+    List<String> names = Arrays.asList("if_class","split_class","split");
+    List<ObjectInspector> fieldOIs = Arrays.<ObjectInspector>asList(
+								    PrimitiveObjectInspectorFactory.javaStringObjectInspector,
+								    PrimitiveObjectInspectorFactory.javaStringObjectInspector,
+								    PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector);
+    ObjectInspector outputOI = ObjectInspectorFactory.getStandardStructObjectInspector(names, fieldOIs);
+    ObjectInspector listOI = ObjectInspectorFactory.getStandardListObjectInspector(outputOI);
+    bos = new ByteArrayOutputStream(1024);
+    dos = new DataOutputStream(bos);
+
+    LOG.debug("done initializing GenericUDFGetSplits");
+    return listOI;
+  }
+
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    retArray.clear();
+
+    String query = stringOI.getPrimitiveJavaObject(arguments[0].get());
+
+    int num = intOI.get(arguments[1].get());
+
+    Driver driver = new Driver();
+    CommandProcessorResponse cpr;
+
+    HiveConf conf = SessionState.get().getConf();
+
+    if (conf == null) {
+      throw new HiveException("Need configuration");
+    }
+
+    LOG.info("setting fetch.task.conversion to none and query file format to \""+LlapOutputFormat.class.toString()+"\"");
+    HiveConf.setVar(conf, ConfVars.HIVEFETCHTASKCONVERSION, "none");
+    HiveConf.setVar(conf, HiveConf.ConfVars.HIVEQUERYRESULTFILEFORMAT, LlapOutputFormat.class.toString());
+
+    cpr = driver.compileAndRespond(query);
+    if(cpr.getResponseCode() != 0) {
+      throw new HiveException("Failed to compile query: "+cpr.getException());
+    }
+
+    QueryPlan plan = driver.getPlan();
+    List<Task<?>> roots = plan.getRootTasks();
+    Schema schema = plan.getResultSchema();
+
+    if (roots == null || roots.size() != 1 || !(roots.get(0) instanceof TezTask)) {
+      throw new HiveException("Was expecting a single TezTask.");
+    }
+
+    Path data = null;
+    InputFormat inp = null;
+    String ifc = null;
+
+    TezWork tezWork = ((TezTask)roots.get(0)).getWork();
+
+    if (tezWork.getAllWork().size() != 1) {
+
+      String tableName = "table_"+UUID.randomUUID().toString().replaceAll("[^A-Za-z0-9 ]", "");
+
+      String ctas = "create temporary table "+tableName+" as "+query;
+      LOG.info("CTAS: "+ctas);
+
+      try {
+        cpr = driver.run(ctas, false);
+      } catch(CommandNeedRetryException e) {
+        throw new HiveException(e);
+      }
+
+      if(cpr.getResponseCode() != 0) {
+        throw new HiveException("Failed to create temp table: " + cpr.getException());
+      }
+
+      query = "select * from " + tableName;
+      cpr = driver.compileAndRespond(query);
+      if(cpr.getResponseCode() != 0) {
+        throw new HiveException("Failed to create temp table: "+cpr.getException());
+      }
+
+      plan = driver.getPlan();
+      roots = plan.getRootTasks();
+      schema = plan.getResultSchema();
+
+      if (roots == null || roots.size() != 1 || !(roots.get(0) instanceof TezTask)) {
+        throw new HiveException("Was expecting a single TezTask.");
+      }
+
+      tezWork = ((TezTask)roots.get(0)).getWork();
+
+      // Table table = db.getTable(tableName);
+      // if (table.isPartitioned()) {
+      //   throw new UDFArgumentException("Table " + tableName + " is partitioned.");
+      // }
+      // data = table.getDataLocation();
+      // LOG.info("looking at: "+data);
+
+      // ifc = table.getInputFormatClass().toString();
+
+      // inp = ReflectionUtils.newInstance(table.getInputFormatClass(), jc);
+    }
+
+    MapWork w = (MapWork)tezWork.getAllWork().get(0);
+    inp = new LlapInputFormat(tezWork, schema);
+    ifc = LlapInputFormat.class.toString();
+
+    try {
+      if (inp instanceof JobConfigurable) {
+        ((JobConfigurable) inp).configure(jc);
+      }
+
+      if (inp instanceof FileInputFormat) {
+        ((FileInputFormat) inp).addInputPath(jc, data);
+      }
+
+      for (InputSplit s: inp.getSplits(jc, num)) {
+        Object[] os = new Object[3];
+        os[0] = ifc;
+        os[1] = s.getClass().toString();
+        bos.reset();
+        s.write(dos);
+        byte[] frozen = bos.toByteArray();
+        os[2] = frozen;
+        retArray.add(os);
+      }
+    } catch(Exception e) {
+      throw new HiveException(e);
+    }
+
+    return retArray;
+  }
+
+  @Override
+  public String getDisplayString(String[] children) {
+    assert children.length == 2;
+    return getStandardDisplayString("get_splits", children);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
new file mode 100644
index 0000000..c49231c
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
@@ -0,0 +1,124 @@
+/**
+ * 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 org.junit.Assert;
+import org.junit.Test;
+import org.junit.Before;
+import org.junit.After;
+
+import java.net.Socket;
+
+import java.io.OutputStream;
+import java.io.InputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.FileInputStream;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.io.RCFile.Reader;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.RecordWriter;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.llap.io.api.LlapProxy;
+
+
+public class TestLlapOutputFormat {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestLlapOutputFormat.class);
+
+  private LlapOutputFormatService service;
+
+  @Before
+  public void setUp() throws IOException {
+    LOG.debug("Setting up output service");
+    service = LlapOutputFormatService.get();
+    LlapProxy.setDaemon(true);
+    LOG.debug("Output service up");
+  }
+
+  @After
+  public void tearDown() throws IOException, InterruptedException {
+    LOG.debug("Tearing down service");
+    service.stop();
+    LOG.debug("Tearing down complete");
+  }
+
+  @Test
+  public void testValues() throws Exception {
+    JobConf job = new JobConf();
+    job.set(LlapOutputFormat.LLAP_OF_ID_KEY, "foobar");
+    LlapOutputFormat format = new LlapOutputFormat();
+
+    HiveConf conf = new HiveConf();
+    Socket socket = new Socket("localhost",
+        conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT));
+
+    LOG.debug("Socket connected");
+
+    socket.getOutputStream().write("foobar".getBytes());
+    socket.getOutputStream().write(0);
+    socket.getOutputStream().flush();
+
+    Thread.sleep(3000);
+
+    LOG.debug("Data written");
+
+    RecordWriter<NullWritable, Text> writer = format.getRecordWriter(null, job, null, null);
+    Text text = new Text();
+
+    LOG.debug("Have record writer");
+
+    for (int i = 0; i < 10; ++i) {
+      text.set(""+i);
+      writer.write(NullWritable.get(),text);
+    }
+
+    writer.close(null);
+
+    InputStream in = socket.getInputStream();
+    RecordReader reader = new LlapRecordReader(in, null, Text.class);
+
+    LOG.debug("Have record reader");
+
+    int count = 0;
+    while(reader.next(NullWritable.get(), text)) {
+      LOG.debug(text.toString());
+      count++;
+    }
+
+    Assert.assertEquals(count,10);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/test/queries/clientpositive/udf_get_splits.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udf_get_splits.q b/ql/src/test/queries/clientpositive/udf_get_splits.q
new file mode 100644
index 0000000..70400e8
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/udf_get_splits.q
@@ -0,0 +1,6 @@
+set hive.fetch.task.conversion=more;
+
+DESCRIBE FUNCTION get_splits;
+DESCRIBE FUNCTION EXTENDED get_splits;
+
+select r.if_class as ic, r.split_class as sc, hash(r.split) as h, length(r.split) as l from (select explode(get_splits("select key, count(*) from src where key % 2 = 0 group by key",5)) as r) t;

http://git-wip-us.apache.org/repos/asf/hive/blob/bf834079/ql/src/test/results/clientpositive/tez/udf_get_splits.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/udf_get_splits.q.out b/ql/src/test/results/clientpositive/tez/udf_get_splits.q.out
new file mode 100644
index 0000000..c8ebe88
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/udf_get_splits.q.out
@@ -0,0 +1,73 @@
+PREHOOK: query: DESCRIBE FUNCTION get_splits
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESCRIBE FUNCTION get_splits
+POSTHOOK: type: DESCFUNCTION
+get_splits(string,int) - Returns an array of length int serialized splits for the referenced tables string.
+PREHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
+POSTHOOK: type: DESCFUNCTION
+get_splits(string,int) - Returns an array of length int serialized splits for the referenced tables string.
+PREHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table_74b4da448d74412d8fc0da37a405efb3
+POSTHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table_74b4da448d74412d8fc0da37a405efb3
+PREHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table_4c8d9e53ea514617a6a72158c9c843c5
+POSTHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table_4c8d9e53ea514617a6a72158c9c843c5
+PREHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table_42dcc08d004e411f8038701980b491e3
+POSTHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table_42dcc08d004e411f8038701980b491e3
+PREHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table_9ef6460ac4a24e4fab1ea09ad94b01e3
+POSTHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table_9ef6460ac4a24e4fab1ea09ad94b01e3
+PREHOOK: query: select r.if_class as ic, r.split_class as sc, hash(r.split) as h, length(r.split) as l from (select explode(get_splits("select key, count(*) from src where key % 2 = 0 group by key",5)) as r) t
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table_bf88922034334ab08a9abb6cf8aa546e
+POSTHOOK: query: select r.if_class as ic, r.split_class as sc, hash(r.split) as h, length(r.split) as l from (select explode(get_splits("select key, count(*) from src where key % 2 = 0 group by key",5)) as r) t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table_bf88922034334ab08a9abb6cf8aa546e
+PREHOOK: query: select r.if_class as ic, r.split_class as sc, hash(r.split) as h, length(r.split) as l from (select explode(get_splits("select key, count(*) from src where key % 2 = 0 group by key",5)) as r) t
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select r.if_class as ic, r.split_class as sc, hash(r.split) as h, length(r.split) as l from (select explode(get_splits("select key, count(*) from src where key % 2 = 0 group by key",5)) as r) t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+class org.apache.hadoop.mapred.TextInputFormat	class org.apache.hadoop.mapred.FileSplit	-1434872849	218
+class org.apache.hadoop.mapred.TextInputFormat	class org.apache.hadoop.mapred.FileSplit	2107621793	218
+class org.apache.hadoop.mapred.TextInputFormat	class org.apache.hadoop.mapred.FileSplit	-1988206222	218
+class org.apache.hadoop.mapred.TextInputFormat	class org.apache.hadoop.mapred.FileSplit	1357774915	218
+class org.apache.hadoop.mapred.TextInputFormat	class org.apache.hadoop.mapred.FileSplit	605302265	218


[38/39] hive git commit: HIVE-13707: Fix TestLlapOutputFormat

Posted by jd...@apache.org.
HIVE-13707: Fix TestLlapOutputFormat


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/89ec219e
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/89ec219e
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/89ec219e

Branch: refs/heads/master
Commit: 89ec219e170d7ce59bf223cdce2fb521ba0a1518
Parents: 763e696
Author: Jason Dere <jd...@hortonworks.com>
Authored: Fri May 6 10:05:49 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Fri May 6 10:05:49 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/llap/TestLlapOutputFormat.java  | 21 +++++++++++---------
 1 file changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/89ec219e/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
index 907d5b0..1d592fb 100644
--- a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hive.llap;
 
 import org.junit.Assert;
 import org.junit.Test;
-import org.junit.Before;
-import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.AfterClass;
 
 import java.net.Socket;
 
@@ -60,18 +60,22 @@ public class TestLlapOutputFormat {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestLlapOutputFormat.class);
 
-  private LlapOutputFormatService service;
+  private static LlapOutputFormatService service;
 
-  @Before
-  public void setUp() throws IOException {
+  @BeforeClass
+  public static void setUp() throws Exception {
     LOG.debug("Setting up output service");
+    Configuration conf = new Configuration();
+    // Pick random avail port
+    HiveConf.setIntVar(conf, HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT, 0);
+    LlapOutputFormatService.initializeAndStart(conf);
     service = LlapOutputFormatService.get();
     LlapProxy.setDaemon(true);
     LOG.debug("Output service up");
   }
 
-  @After
-  public void tearDown() throws IOException, InterruptedException {
+  @AfterClass
+  public static void tearDown() throws IOException, InterruptedException {
     LOG.debug("Tearing down service");
     service.stop();
     LOG.debug("Tearing down complete");
@@ -87,8 +91,7 @@ public class TestLlapOutputFormat {
       LlapOutputFormat format = new LlapOutputFormat();
 
       HiveConf conf = new HiveConf();
-      Socket socket = new Socket("localhost",
-          conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT));
+      Socket socket = new Socket("localhost", service.getPort());
 
       LOG.debug("Socket connected");
 


[06/39] hive git commit: HIVE-13162: Fixes for LlapDump and FileSinkoperator

Posted by jd...@apache.org.
HIVE-13162: Fixes for LlapDump and FileSinkoperator


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/57761e34
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/57761e34
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/57761e34

Branch: refs/heads/master
Commit: 57761e34ffec80487a0e678bea6d2dc87bfc9b11
Parents: 7c4b3ff
Author: Gunther Hagleitner <gu...@apache.org>
Authored: Thu Feb 25 16:54:52 2016 -0800
Committer: Gunther Hagleitner <gu...@apache.org>
Committed: Thu Feb 25 17:20:38 2016 -0800

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |    4 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |    3 +-
 .../src/java/org/apache/hive/jdbc/LlapDump.java |   16 +-
 .../org/apache/hive/jdbc/LlapInputFormat.java   |    4 +-
 .../hadoop/hive/llap/LlapInputFormat.java       |   56 +-
 .../llap/daemon/impl/ContainerRunnerImpl.java   |   10 +-
 .../ext/LlapTaskUmbilicalExternalClient.java    |    8 -
 .../llap/tezplugins/LlapTaskCommunicator.java   |    2 +-
 .../tezplugins/LlapTaskSchedulerService.java    |    4 +-
 .../hadoop/hive/llap/LlapInputFormat.java       |  159 --
 .../hive/llap/LlapOutputFormatService.java      |   77 +-
 .../hadoop/hive/llap/LlapRecordWriter.java      |   10 +-
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |   13 +
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |    4 +-
 .../hive/ql/exec/tez/HiveSplitGenerator.java    |    1 +
 .../hive/ql/exec/tez/MapRecordProcessor.java    |    4 +-
 .../hive/ql/io/HivePassThroughRecordWriter.java |    4 +
 .../ql/udf/generic/GenericUDFGetSplits.java     |  412 ----
 .../udf/generic/GenericUDTFExecuteSplits.java   |  124 +
 .../ql/udf/generic/GenericUDTFGetSplits.java    |  420 ++++
 .../org/apache/tez/dag/api/TaskSpecBuilder.java |    4 +-
 .../hadoop/hive/llap/TestLlapOutputFormat.java  |   62 +-
 .../queries/clientpositive/udf_get_splits.q     |    6 -
 .../queries/clientpositive/udtf_get_splits.q    |   43 +
 .../clientpositive/llap/udtf_get_splits.q.out   | 2130 ++++++++++++++++++
 25 files changed, 2874 insertions(+), 706 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index deb9905..13b5113 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -435,7 +435,6 @@ minitez.query.files=bucket_map_join_tez1.q,\
   tez_smb_main.q,\
   tez_smb_1.q,\
   tez_smb_empty.q,\
-  udf_get_splits.q,\
   vector_join_part_col_char.q,\
   vectorized_dynamic_partition_pruning.q,\
   tez_multi_union.q,\
@@ -495,7 +494,8 @@ minillap.query.files=bucket_map_join_tez1.q,\
   vectorized_dynamic_partition_pruning.q,\
   tez_multi_union.q,\
   tez_join.q,\
-  tez_union_multiinsert.q
+  tez_union_multiinsert.q,\
+  udtf_get_splits.q
 
 encrypted.query.files=encryption_join_unencrypted_tbl.q,\
   encryption_insert_partition_static.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 5e81e98..e524bd2 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -1460,7 +1460,8 @@ public class QTestUtil {
       ".*Input:.*/data/files/.*",
       ".*Output:.*/data/files/.*",
       ".*total number of created files now is.*",
-      ".*.hive-staging.*"
+      ".*.hive-staging.*",
+      "table_.*"
   });
 
   private final Pattern[] partialReservedPlanMask = toPattern(new String[] {

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java b/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
index a807f6c..7ed0a0e 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
@@ -24,6 +24,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.FileInputStream;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
 
@@ -61,6 +62,7 @@ public class LlapDump {
   private static String user = "hive";
   private static String pwd = "";
   private static String query = "select * from test";
+  private static String numSplits = "1";
 
   public static void main(String[] args) throws Exception {
     Options opts = createOptions();
@@ -84,6 +86,10 @@ public class LlapDump {
       pwd = cli.getOptionValue("p");
     }
 
+    if (cli.hasOption('n')) {
+      numSplits = cli.getOptionValue("n");
+    }
+
     if (cli.getArgs().length > 0) {
       query = cli.getArgs()[0];
     }
@@ -95,7 +101,7 @@ public class LlapDump {
     LlapInputFormat format = new LlapInputFormat(url, user, pwd, query);
     JobConf job = new JobConf();
 
-    InputSplit[] splits = format.getSplits(job, 1);
+    InputSplit[] splits = format.getSplits(job, Integer.parseInt(numSplits));
 
     if (splits.length == 0) {
       System.out.println("No splits returned - empty scan");
@@ -104,6 +110,7 @@ public class LlapDump {
       boolean first = true;
 
       for (InputSplit s: splits) {
+        LOG.info("Processing input split s from " + Arrays.toString(s.getLocations()));
         RecordReader<NullWritable, Text> reader = format.getRecordReader(s, job, null);
 
         if (reader instanceof LlapRecordReader && first) {
@@ -122,6 +129,7 @@ public class LlapDump {
           System.out.println(value);
         }
       }
+      System.exit(0);
     }
   }
 
@@ -146,6 +154,12 @@ public class LlapDump {
         .hasArg()
         .create('p'));
 
+    result.addOption(OptionBuilder
+        .withLongOpt("num")
+        .withDescription("number of splits")
+        .hasArg()
+        .create('n'));
+
     return result;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java b/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
index 5af2175..9a7c16d 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
@@ -64,6 +64,8 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
   public final String USER_KEY = "llap.if.user";
   public final String PWD_KEY = "llap.if.pwd";
 
+  public final String SPLIT_QUERY = "select get_splits(\"%s\",%d)";
+
   private Connection con;
   private Statement stmt;
 
@@ -105,7 +107,7 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     try {
       con = DriverManager.getConnection(url,user,pwd);
       stmt = con.createStatement();
-      String sql = "select r.if_class as ic, r.split_class as sc, r.split as s from (select explode(get_splits(\""+query+"\","+numSplits+")) as r) t";
+      String sql = String.format(SPLIT_QUERY, query, numSplits);
       ResultSet res = stmt.executeQuery(sql);
       while (res.next()) {
         // deserialize split

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
index d8066d5..b32d662 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
@@ -67,7 +67,7 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
    */
   @Override
   public RecordReader<NullWritable, V> getRecordReader(InputSplit split, JobConf job,
-                                                       Reporter reporter) throws IOException {
+      Reporter reporter) throws IOException {
 
     LlapInputSplit llapSplit = (LlapInputSplit) split;
     SubmitWorkInfo submitWorkInfo = SubmitWorkInfo.fromBytes(llapSplit.getPlanBytes());
@@ -75,22 +75,15 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     // TODO HACK: Spark is built with Hive-1.2.1, does not have access to HiveConf.ConfVars.LLAP_DAEMON_RPC_PORT
     int llapSubmitPort = job.getInt("hive.llap.daemon.rpc.port", 15001);
 
-    LOG.info("ZZZ: DBG: Starting LlapTaskUmbilicalExternalClient");
-
     LlapTaskUmbilicalExternalClient llapClient =
-        new LlapTaskUmbilicalExternalClient(job, submitWorkInfo.getTokenIdentifier(),
-            submitWorkInfo.getToken());
+      new LlapTaskUmbilicalExternalClient(job, submitWorkInfo.getTokenIdentifier(),
+          submitWorkInfo.getToken());
     llapClient.init(job);
     llapClient.start();
 
-    LOG.info("ZZZ: DBG: Crated LlapClient");
-    // TODO KKK Shutdown the llap client.
-
     SubmitWorkRequestProto submitWorkRequestProto =
-        constructSubmitWorkRequestProto(submitWorkInfo, llapSplit.getSplitNum(),
-            llapClient.getAddress(), submitWorkInfo.getToken());
-
-    LOG.info("ZZZ: DBG: Created submitWorkRequest for: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString());
+      constructSubmitWorkRequestProto(submitWorkInfo, llapSplit.getSplitNum(),
+          llapClient.getAddress(), submitWorkInfo.getToken());
 
     TezEvent tezEvent = new TezEvent();
     DataInputBuffer dib = new DataInputBuffer();
@@ -116,7 +109,7 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     socket.getOutputStream().write(0);
     socket.getOutputStream().flush();
 
-    LOG.debug("Registered id: " + id);
+    LOG.info("Registered id: " + id);
 
     return new LlapRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class);
   }
@@ -127,16 +120,16 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
   }
 
   private SubmitWorkRequestProto constructSubmitWorkRequestProto(SubmitWorkInfo submitWorkInfo,
-                                                                 int taskNum,
-                                                                 InetSocketAddress address,
-                                                                 Token<JobTokenIdentifier> token) throws
-      IOException {
+      int taskNum,
+      InetSocketAddress address,
+      Token<JobTokenIdentifier> token) throws
+        IOException {
     TaskSpec taskSpec = submitWorkInfo.getTaskSpec();
     ApplicationId appId = submitWorkInfo.getFakeAppId();
 
     SubmitWorkRequestProto.Builder builder = SubmitWorkRequestProto.newBuilder();
     // This works, assuming the executor is running within YARN.
-    LOG.info("DBG: Setting user in submitWorkRequest to: " +
+    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());
@@ -144,10 +137,9 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     builder.setTokenIdentifier(appId.toString());
 
     ContainerId containerId =
-        ContainerId.newInstance(ApplicationAttemptId.newInstance(appId, 0), taskNum);
+      ContainerId.newInstance(ApplicationAttemptId.newInstance(appId, 0), taskNum);
     builder.setContainerIdString(containerId.toString());
 
-
     builder.setAmHost(address.getHostName());
     builder.setAmPort(address.getPort());
     Credentials taskCredentials = new Credentials();
@@ -155,18 +147,18 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     // 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));
+    //    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);

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/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 e80fb15..d47355a 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
@@ -265,10 +265,10 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
     List<QueryFragmentInfo> knownFragments =
         queryTracker
             .queryComplete(queryIdentifier, request.getDeleteDelay());
-    LOG.info("DBG: Pending fragment count for completed query {} = {}", queryIdentifier,
+    LOG.info("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());
     }
@@ -277,7 +277,7 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
 
   @Override
   public TerminateFragmentResponseProto terminateFragment(TerminateFragmentRequestProto request) {
-    LOG.info("DBG: Received terminateFragment request for {}", request.getFragmentIdentifierString());
+    LOG.info("Received terminateFragment request for {}", request.getFragmentIdentifierString());
     executorService.killFragment(request.getFragmentIdentifierString());
     return TerminateFragmentResponseProto.getDefaultInstance();
   }
@@ -356,10 +356,10 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
     LOG.info("Processing query failed notification for {}", queryIdentifier);
     List<QueryFragmentInfo> knownFragments =
         queryTracker.queryComplete(queryIdentifier, -1);
-    LOG.info("DBG: Pending fragment count for failed query {} = {}", queryIdentifier,
+    LOG.info("Pending fragment count for failed query {} = {}", queryIdentifier,
         knownFragments.size());
     for (QueryFragmentInfo fragmentInfo : knownFragments) {
-      LOG.info("DBG: Issuing killFragment for failed query {} {}", queryIdentifier,
+      LOG.info("Issuing killFragment for failed query {} {}", queryIdentifier,
           fragmentInfo.getFragmentIdentifierString());
       executorService.killFragment(fragmentInfo.getFragmentIdentifierString());
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java b/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
index 4305682..16cfd01 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
@@ -88,12 +88,6 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
   public void submitWork(final SubmitWorkRequestProto submitWorkRequestProto, String llapHost, int llapPort, List<TezEvent> tezEvents) {
     Preconditions.checkArgument(submitWorkRequestProto.getUsingTezAm() == false);
 
-
-    LOG.warn("ZZZ: DBG: " + " Submitting fragment: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString() + " on host: " + llapHost + ", port=" + llapPort);
-//    LOG.info("ZZZ: DBG: " + " Complete SubmitWorkRequest: " + submitWorkRequestProto);
-//    submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString()
-
-    LOG.info("ZZZ: DBG: Received {} events for {}", tezEvents.size(), submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString());
     // Register the pending events to be sent for this spec.
     pendingEvents.putIfAbsent(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), tezEvents);
 
@@ -109,7 +103,6 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
                 return;
               }
             }
-            LOG.info("DBG: Submitted " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString());
           }
 
           @Override
@@ -166,7 +159,6 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
       TezHeartbeatResponse response = new TezHeartbeatResponse();
       // Assuming TaskAttemptId and FragmentIdentifierString are the same. Verify this.
       TezTaskAttemptID taskAttemptId = request.getCurrentTaskAttemptID();
-      LOG.info("ZZZ: DBG: Received heartbeat from taskAttemptId: " + taskAttemptId.toString());
 
       List<TezEvent> tezEvents = pendingEvents.remove(taskAttemptId.toString());
       if (tezEvents == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/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 76d095a..91e4323 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
@@ -349,7 +349,7 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
   private void sendTaskTerminated(final TezTaskAttemptID taskAttemptId,
                                   boolean invokedByContainerEnd) {
     LOG.info(
-        "DBG: Attempting to send terminateRequest for fragment {} due to internal preemption invoked by {}",
+        "Attempting to send terminateRequest for fragment {} due to internal preemption invoked by {}",
         taskAttemptId.toString(), invokedByContainerEnd ? "containerEnd" : "taskEnd");
     LlapNodeId nodeId = entityTracker.getNodeIdForTaskAttempt(taskAttemptId);
     // NodeId can be null if the task gets unregistered due to failure / being killed by the daemon itself

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
index 3bca0da..e1ad12d 100644
--- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
@@ -931,7 +931,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
           }
         } else {
           // No tasks qualify as preemptable
-          LOG.info("DBG: No tasks qualify as killable to schedule tasks at priority {}", forPriority);
+          LOG.info("No tasks qualify as killable to schedule tasks at priority {}", forPriority);
           break;
         }
       }
@@ -941,7 +941,7 @@ public class LlapTaskSchedulerService extends TaskScheduler {
     // Send out the preempted request outside of the lock.
     if (preemptedTaskList != null) {
       for (TaskInfo taskInfo : preemptedTaskList) {
-        LOG.info("DBG: Preempting task {}", taskInfo);
+        LOG.info("Preempting task {}", taskInfo);
         getContext().preemptContainer(taskInfo.containerId);
         // Preemption will finally be registered as a deallocateTask as a result of preemptContainer
         // That resets preemption info and allows additional tasks to be pre-empted if required.

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
deleted file mode 100644
index d308ec8..0000000
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.llap;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.Set;
-
-import javax.security.auth.login.LoginException;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.exec.tez.TezProcessor;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.dag.api.TaskSpecBuilder;
-import org.apache.tez.runtime.api.impl.TaskSpec;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.Socket;
-
-import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
-
-import com.esotericsoftware.kryo.Kryo;
-import java.io.ByteArrayOutputStream;
-import java.io.OutputStream;
-import java.io.InputStream;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URISyntaxException;
-import java.io.FileNotFoundException;
-import java.util.UUID;
-
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.commons.io.FilenameUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapred.SplitLocationInfo;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.tez.dag.api.DAG;
-import org.apache.tez.dag.api.Vertex;
-import org.apache.tez.runtime.api.Event;
-import org.apache.hadoop.hive.ql.Context;
-import org.apache.hadoop.hive.ql.exec.tez.DagUtils;
-import org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator;
-import org.apache.hadoop.hive.ql.plan.MapWork;
-import org.apache.hadoop.hive.ql.plan.TezWork;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.Schema;
-import org.apache.tez.dag.api.TaskLocationHint;
-import org.apache.tez.dag.api.VertexLocationHint;
-import org.apache.tez.dag.api.event.VertexStateUpdate;
-import org.apache.tez.mapreduce.hadoop.InputSplitInfoMem;
-import org.apache.tez.mapreduce.hadoop.MRInputHelpers;
-import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRInputUserPayloadProto;
-import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitProto;
-import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitsProto;
-import org.apache.tez.runtime.api.Event;
-import org.apache.tez.runtime.api.InputInitializer;
-import org.apache.tez.runtime.api.InputInitializerContext;
-import org.apache.tez.runtime.api.InputSpecUpdate;
-import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
-import org.apache.tez.runtime.api.events.InputDataInformationEvent;
-import org.apache.tez.runtime.api.events.InputInitializerEvent;
-
-import com.google.common.base.Preconditions;
-
-public class LlapInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
-
-  private static final Logger LOG = LoggerFactory.getLogger(LlapInputFormat.class);
-
-  private final TezWork work;
-  private final Schema schema;
-
-  public LlapInputFormat(TezWork tezWork, Schema schema) {
-    this.work = tezWork;
-    this.schema = schema;
-  }
-
-  // need empty constructor for bean instantiation
-  public LlapInputFormat() {
-    // None of these fields should be required during getRecordReader,
-    // and should not be read.
-    work = null;
-    schema = null;
-  }
-
-  /*
-   * This proxy record reader has the duty of establishing a connected socket with LLAP, then fire
-   * off the work in the split to LLAP and finally return the connected socket back in an
-   * LlapRecordReader. The LlapRecordReader class reads the results from the socket.
-   */
-  @Override
-  public RecordReader<NullWritable, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
-
-    // Calls a static method to ensure none of the object fields are read.
-    return _getRecordReader(split, job, reporter);
-  }
-
-  private static RecordReader _getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws
-      IOException {
-    LlapInputSplit llapSplit = (LlapInputSplit)split;
-
-    // TODO: push event into LLAP
-
-    // this is just the portion that sets up the io to receive data
-    String host = split.getLocations()[0];
-
-    // TODO: need to construct id here. Format is queryId + "_" + taskIndex
-    String id = "foobar";
-
-    HiveConf conf = new HiveConf();
-    Socket socket = new Socket(host,
-        conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT));
-
-    LOG.debug("Socket connected");
-
-    socket.getOutputStream().write(id.getBytes());
-    socket.getOutputStream().write(0);
-    socket.getOutputStream().flush();
-
-    LOG.debug("Registered id: " + id);
-
-    return new LlapRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class);
-  }
-
-  @Override
-  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
-    throw new IOException("These are not the splits you are looking for.");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
index 4f38ff1..a197d7b 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
@@ -81,45 +81,45 @@ public class LlapOutputFormatService {
 
   public void start() throws IOException {
     executor.submit(new Runnable() {
-      byte[] buffer = new byte[4096];
-      @Override
-      public void run() {
-	while (true) {
-	  Socket s = null;
-	  try {
-	    s = socket.accept();
-	    String id = readId(s);
-	    LOG.debug("Received: "+id);
-	    registerReader(s, id);
-	  } catch (IOException io) {
-	    if (s != null) {
-	      try{
-		s.close();
-	      } catch (IOException io2) {
-		// ignore
-	      }
-	    }
-	  }
-	}
-      }
+        byte[] buffer = new byte[4096];
+        @Override
+        public void run() {
+          while (true) {
+            Socket s = null;
+            try {
+              s = socket.accept();
+              String id = readId(s);
+              LOG.debug("Received: "+id);
+              registerReader(s, id);
+            } catch (IOException io) {
+              if (s != null) {
+                try{
+                  s.close();
+                } catch (IOException io2) {
+                  // ignore
+                }
+              }
+            }
+          }
+        }
 
-    private String readId(Socket s) throws IOException {
-      InputStream in = s.getInputStream();
-      int idx = 0;
-      while((buffer[idx++] = (byte)in.read()) != '\0') {}
-      return new String(buffer,0,idx-1);
-    }
+        private String readId(Socket s) throws IOException {
+          InputStream in = s.getInputStream();
+          int idx = 0;
+          while((buffer[idx++] = (byte)in.read()) != '\0') {}
+          return new String(buffer,0,idx-1);
+        }
 
-    private void registerReader(Socket s, String id) throws IOException {
-      synchronized(service) {
-	LOG.debug("registering socket for: "+id);
-	LlapRecordWriter writer = new LlapRecordWriter(s.getOutputStream());
-        writers.put(id, writer);
-        service.notifyAll();
+        private void registerReader(Socket s, String id) throws IOException {
+          synchronized(service) {
+            LOG.debug("registering socket for: "+id);
+            LlapRecordWriter writer = new LlapRecordWriter(s.getOutputStream());
+            writers.put(id, writer);
+            service.notifyAll();
+          }
+        }
       }
-    }
-    }
-    );
+      );
   }
 
   public void stop() throws IOException, InterruptedException {
@@ -132,10 +132,11 @@ public class LlapOutputFormatService {
     RecordWriter writer = null;
     synchronized(service) {
       while ((writer = writers.get(id)) == null) {
-	LOG.debug("Waiting for writer for: "+id);
-	service.wait();
+        LOG.info("Waiting for writer for: "+id);
+        service.wait();
       }
     }
+    LOG.info("Returning writer for: "+id);
     return writer;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordWriter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordWriter.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordWriter.java
index 4d1996c..b632fae 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordWriter.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordWriter.java
@@ -20,19 +20,18 @@ package org.apache.hadoop.hive.llap;
 
 import java.io.IOException;
 import java.io.OutputStream;
-import java.io.DataOutputStream;;
+import java.io.DataOutputStream;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.mapred.RecordWriter;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.JobConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class LlapRecordWriter<K extends Writable, V extends WritableComparable>
   implements RecordWriter<K,V> {
+  public static final Logger LOG = LoggerFactory.getLogger(LlapRecordWriter.class);
 
   DataOutputStream dos;
 
@@ -42,6 +41,7 @@ public class LlapRecordWriter<K extends Writable, V extends WritableComparable>
 
   @Override
   public void close(Reporter reporter) throws IOException {
+    LOG.info("CLOSING the record writer output stream");
     dos.close();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index 0899793..02439be 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@ -36,11 +36,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.llap.LlapOutputFormat;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
@@ -202,6 +204,17 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
         }
       }
       try {
+        if ("org.apache.hadoop.hive.llap.LlapStorageHandler".equals(getConf().getTableInfo().getProperties().
+            get(hive_metastoreConstants.META_TABLE_STORAGE))) {
+          (new LlapOutputFormat())
+              .getRecordWriter(null,
+                  hconf instanceof JobConf ? (JobConf) hconf : new JobConf(hconf), null, null)
+              .close(null);
+        }
+      } catch (IOException e) {
+        // ignored
+      }
+      try {
         for (int i = 0; i < updaters.length; i++) {
           if (updaters[i] != null) {
             updaters[i].close(abort);

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index f3afa24..c782466 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -344,8 +344,6 @@ public final class FunctionRegistry {
     system.registerGenericUDF("ewah_bitmap_or", GenericUDFEWAHBitmapOr.class);
     system.registerGenericUDF("ewah_bitmap_empty", GenericUDFEWAHBitmapEmpty.class);
 
-    system.registerGenericUDF("get_splits", GenericUDFGetSplits.class);
-
     // Aliases for Java Class Names
     // These are used in getImplicitConvertUDFMethod
     system.registerUDF(serdeConstants.BOOLEAN_TYPE_NAME, UDFToBoolean.class, false, UDFToBoolean.class.getSimpleName());
@@ -444,6 +442,8 @@ public final class FunctionRegistry {
     system.registerGenericUDTF("parse_url_tuple", GenericUDTFParseUrlTuple.class);
     system.registerGenericUDTF("posexplode", GenericUDTFPosExplode.class);
     system.registerGenericUDTF("stack", GenericUDTFStack.class);
+    system.registerGenericUDTF("get_splits", GenericUDTFGetSplits.class);
+    system.registerGenericUDTF("execute_splits", GenericUDTFExecuteSplits.class);
 
     //PTF declarations
     system.registerGenericUDF(LEAD_FUNC_NAME, GenericUDFLead.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
index 011e459..b16368f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
@@ -92,6 +92,7 @@ public class HiveSplitGenerator extends InputInitializer {
 
     this.conf = conf;
     this.work = work;
+    this.jobConf = new JobConf(conf);
 
     // TODO RSHACK - assuming grouping enabled always.
     userPayloadProto = MRInputUserPayloadProto.newBuilder().setGroupingEnabled(true).build();

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
index 3fe70ab..7a3d6a6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
@@ -95,7 +95,9 @@ public class MapRecordProcessor extends RecordProcessor {
     super(jconf, context);
     String queryId = HiveConf.getVar(jconf, HiveConf.ConfVars.HIVEQUERYID);
     if (LlapProxy.isDaemon()) { // do not cache plan
-      jconf.set(LlapOutputFormat.LLAP_OF_ID_KEY, queryId + "_" + context.getTaskIndex());
+      String id = queryId + "_" + context.getTaskIndex();
+      l4j.info("LLAP_OF_ID: "+id);
+      jconf.set(LlapOutputFormat.LLAP_OF_ID_KEY, id);
       cache = new org.apache.hadoop.hive.ql.exec.mr.ObjectCache();
     } else {
       cache = ObjectCacheFactory.getCache(jconf, queryId);

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java
index 454c321..6d00a0a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java
@@ -23,11 +23,14 @@ import java.io.IOException;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 public class HivePassThroughRecordWriter <K extends WritableComparable<?>, V extends Writable>
 implements RecordWriter {
 
+  public static final Logger LOG = LoggerFactory.getLogger(HivePassThroughRecordWriter.class);
   private final org.apache.hadoop.mapred.RecordWriter<K, V> mWriter;
 
   public HivePassThroughRecordWriter(org.apache.hadoop.mapred.RecordWriter<K, V> writer) {
@@ -42,6 +45,7 @@ implements RecordWriter {
 
   @Override
   public void close(boolean abort) throws IOException {
+    LOG.info("Closing the pass through writer.");
     //close with null reporter
     mWriter.close(null);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
deleted file mode 100644
index f69dea3..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
+++ /dev/null
@@ -1,412 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.udf.generic;
-
-import javax.security.auth.login.LoginException;
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Random;
-import java.util.Set;
-import java.util.UUID;
-
-import com.esotericsoftware.kryo.Kryo;
-import com.google.common.base.Preconditions;
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.commons.io.FilenameUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.llap.LlapInputSplit;
-import org.apache.hadoop.hive.llap.LlapOutputFormat;
-import org.apache.hadoop.hive.llap.SubmitWorkInfo;
-import org.apache.hadoop.hive.metastore.api.Schema;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Context;
-import org.apache.hadoop.hive.ql.Driver;
-import org.apache.hadoop.hive.ql.QueryPlan;
-import org.apache.hadoop.hive.ql.exec.Description;
-import org.apache.hadoop.hive.ql.exec.MapredContext;
-import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
-import org.apache.hadoop.hive.ql.exec.tez.DagUtils;
-import org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator;
-import org.apache.hadoop.hive.ql.exec.tez.TezTask;
-import org.apache.hadoop.hive.ql.metadata.Hive;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.plan.MapWork;
-import org.apache.hadoop.hive.ql.plan.TezWork;
-import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.ql.udf.UDFType;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.SplitLocationInfo;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.tez.dag.api.DAG;
-import org.apache.tez.dag.api.TaskLocationHint;
-import org.apache.tez.dag.api.TaskSpecBuilder;
-import org.apache.tez.dag.api.Vertex;
-import org.apache.tez.runtime.api.Event;
-import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
-import org.apache.tez.runtime.api.impl.EventMetaData;
-import org.apache.tez.runtime.api.impl.TaskSpec;
-import org.apache.tez.runtime.api.impl.TezEvent;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * GenericUDFGetSplits.
- *
- */
-@Description(name = "get_splits", value = "_FUNC_(string,int) - "
-    + "Returns an array of length int serialized splits for the referenced tables string.")
-@UDFType(deterministic = false)
-public class GenericUDFGetSplits extends GenericUDF {
-
-  private static final Logger LOG = LoggerFactory.getLogger(GenericUDFGetSplits.class);
-
-  private static final String LLAP_INTERNAL_INPUT_FORMAT_NAME = "org.apache.hadoop.hive.llap.LlapInputFormat";
-
-  private transient StringObjectInspector stringOI;
-  private transient IntObjectInspector intOI;
-  private final ArrayList<Object> retArray = new ArrayList<Object>();
-  private transient JobConf jc;
-  private transient Hive db;
-  private ByteArrayOutputStream bos;
-  private DataOutput dos;
-
-  @Override
-  public ObjectInspector initialize(ObjectInspector[] arguments)
-    throws UDFArgumentException {
-
-    LOG.debug("initializing GenericUDFGetSplits");
-
-    try {
-      if (SessionState.get() != null && SessionState.get().getConf() != null) {
-        HiveConf conf = SessionState.get().getConf();
-        jc = DagUtils.getInstance().createConfiguration(conf);
-        db = Hive.get(conf);
-      } else {
-        jc = MapredContext.get().getJobConf();
-        db = Hive.get();
-      }
-    } catch(Exception e) {
-      LOG.error("Failed to initialize: ",e);
-      throw new UDFArgumentException(e);
-    }
-
-    LOG.debug("Initialized conf, jc and metastore connection");
-
-    if (arguments.length != 2) {
-      throw new UDFArgumentLengthException("The function GET_SPLITS accepts 2 arguments.");
-    } else if (!(arguments[0] instanceof StringObjectInspector)) {
-      LOG.error("Got "+arguments[0].getTypeName()+" instead of string.");
-      throw new UDFArgumentTypeException(0, "\""
-          + "string\" is expected at function GET_SPLITS, " + "but \""
-          + arguments[0].getTypeName() + "\" is found");
-    } else if (!(arguments[1] instanceof IntObjectInspector)) {
-      LOG.error("Got "+arguments[1].getTypeName()+" instead of int.");
-      throw new UDFArgumentTypeException(1, "\""
-          + "int\" is expected at function GET_SPLITS, " + "but \""
-          + arguments[1].getTypeName() + "\" is found");
-    }
-
-    stringOI = (StringObjectInspector) arguments[0];
-    intOI = (IntObjectInspector) arguments[1];
-
-    List<String> names = Arrays.asList("if_class","split_class","split");
-    List<ObjectInspector> fieldOIs = Arrays.<ObjectInspector>asList(
-                                                                    PrimitiveObjectInspectorFactory.javaStringObjectInspector,
-                                                                    PrimitiveObjectInspectorFactory.javaStringObjectInspector,
-                                                                    PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector);
-    ObjectInspector outputOI = ObjectInspectorFactory.getStandardStructObjectInspector(names, fieldOIs);
-    ObjectInspector listOI = ObjectInspectorFactory.getStandardListObjectInspector(outputOI);
-    bos = new ByteArrayOutputStream(1024);
-    dos = new DataOutputStream(bos);
-
-    LOG.debug("done initializing GenericUDFGetSplits");
-    return listOI;
-  }
-
-  @Override
-  public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    retArray.clear();
-
-    String query = stringOI.getPrimitiveJavaObject(arguments[0].get());
-
-    int num = intOI.get(arguments[1].get());
-
-    Driver driver = new Driver();
-    CommandProcessorResponse cpr;
-
-    HiveConf conf = SessionState.get().getConf();
-
-    if (conf == null) {
-      throw new HiveException("Need configuration");
-    }
-
-    String fetchTaskConversion = HiveConf.getVar(conf, ConfVars.HIVEFETCHTASKCONVERSION);
-    String queryResultFormat = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYRESULTFILEFORMAT);
-
-    try {
-      LOG.info("setting fetch.task.conversion to none and query file format to \""+LlapOutputFormat.class.getName()+"\"");
-      HiveConf.setVar(conf, ConfVars.HIVEFETCHTASKCONVERSION, "none");
-      HiveConf.setVar(conf, HiveConf.ConfVars.HIVEQUERYRESULTFILEFORMAT, "Llap");
-
-      cpr = driver.compileAndRespond(query);
-      if(cpr.getResponseCode() != 0) {
-        throw new HiveException("Failed to compile query: "+cpr.getException());
-      }
-
-      QueryPlan plan = driver.getPlan();
-      List<Task<?>> roots = plan.getRootTasks();
-      Schema schema = plan.getResultSchema();
-
-      if (roots == null || roots.size() != 1 || !(roots.get(0) instanceof TezTask)) {
-        throw new HiveException("Was expecting a single TezTask.");
-      }
-
-      TezWork tezWork = ((TezTask)roots.get(0)).getWork();
-
-      if (tezWork.getAllWork().size() != 1) {
-
-        String tableName = "table_"+UUID.randomUUID().toString().replaceAll("[^A-Za-z0-9 ]", "");
-
-        String ctas = "create temporary table "+tableName+" as "+query;
-        LOG.info("CTAS: "+ctas);
-
-        try {
-          cpr = driver.run(ctas, false);
-        } catch(CommandNeedRetryException e) {
-          throw new HiveException(e);
-        }
-
-        if(cpr.getResponseCode() != 0) {
-          throw new HiveException("Failed to create temp table: " + cpr.getException());
-        }
-
-        query = "select * from " + tableName;
-        cpr = driver.compileAndRespond(query);
-        if(cpr.getResponseCode() != 0) {
-          throw new HiveException("Failed to create temp table: "+cpr.getException());
-        }
-
-        plan = driver.getPlan();
-        roots = plan.getRootTasks();
-        schema = plan.getResultSchema();
-
-        if (roots == null || roots.size() != 1 || !(roots.get(0) instanceof TezTask)) {
-          throw new HiveException("Was expecting a single TezTask.");
-        }
-
-        tezWork = ((TezTask)roots.get(0)).getWork();
-      }
-
-      MapWork w = (MapWork)tezWork.getAllWork().get(0);
-
-      try {
-        for (InputSplit s: getSplits(jc, num, tezWork, schema)) {
-          Object[] os = new Object[3];
-          os[0] = LLAP_INTERNAL_INPUT_FORMAT_NAME;
-          os[1] = s.getClass().getName();
-          bos.reset();
-          s.write(dos);
-          byte[] frozen = bos.toByteArray();
-          os[2] = frozen;
-          retArray.add(os);
-        }
-      } catch(Exception e) {
-        throw new HiveException(e);
-      }
-    } finally {
-      HiveConf.setVar(conf, ConfVars.HIVEFETCHTASKCONVERSION, fetchTaskConversion);
-      HiveConf.setVar(conf, HiveConf.ConfVars.HIVEQUERYRESULTFILEFORMAT, queryResultFormat);
-    }
-    return retArray;
-  }
-
-  public InputSplit[] getSplits(JobConf job, int numSplits, TezWork work, Schema schema) throws IOException {
-    DAG dag = DAG.create(work.getName());
-    dag.setCredentials(job.getCredentials());
-    // TODO: set access control? TezTask.setAccessControlsForCurrentUser(dag);
-
-    DagUtils utils = DagUtils.getInstance();
-    Context ctx = new Context(job);
-    MapWork mapWork = (MapWork) work.getAllWork().get(0);
-    // bunch of things get setup in the context based on conf but we need only the MR tmp directory
-    // for the following method.
-    JobConf wxConf = utils.initializeVertexConf(job, ctx, mapWork);
-    Path scratchDir = utils.createTezDir(ctx.getMRScratchDir(), job);
-    FileSystem fs = scratchDir.getFileSystem(job);
-    try {
-      LocalResource appJarLr = createJarLocalResource(utils.getExecJarPathLocal(), utils, job);
-      Vertex wx = utils.createVertex(wxConf, mapWork, scratchDir, appJarLr,
-          new ArrayList<LocalResource>(), fs, ctx, false, work,
-          work.getVertexType(mapWork));
-      String vertexName = wx.getName();
-      dag.addVertex(wx);
-      utils.addCredentials(mapWork, dag);
-
-
-      // we have the dag now proceed to get the splits:
-      HiveSplitGenerator splitGenerator = new HiveSplitGenerator(null);
-      Preconditions.checkState(HiveConf.getBoolVar(wxConf,
-          HiveConf.ConfVars.HIVE_TEZ_GENERATE_CONSISTENT_SPLITS));
-      Preconditions.checkState(HiveConf.getBoolVar(wxConf,
-          HiveConf.ConfVars.LLAP_CLIENT_CONSISTENT_SPLITS));
-      splitGenerator.initializeSplitGenerator(wxConf, mapWork);
-      List<Event> eventList = splitGenerator.initialize();
-
-      // hack - just serializing with kryo for now. This needs to be done properly
-      InputSplit[] result = new InputSplit[eventList.size() - 1];
-      DataOutputBuffer dob = new DataOutputBuffer();
-
-      InputConfigureVertexTasksEvent configureEvent = (InputConfigureVertexTasksEvent) eventList.get(0);
-
-      List<TaskLocationHint> hints = configureEvent.getLocationHint().getTaskLocationHints();
-
-      Preconditions.checkState(hints.size() == eventList.size() -1);
-
-      LOG.error("DBG: NumEvents=" + eventList.size());
-      LOG.error("DBG: NumSplits=" + result.length);
-
-      ApplicationId fakeApplicationId = ApplicationId.newInstance(Math.abs(new Random().nextInt()), 0);
-      TaskSpec taskSpec =
-          new TaskSpecBuilder().constructTaskSpec(dag, vertexName, eventList.size() - 1, fakeApplicationId);
-
-      SubmitWorkInfo submitWorkInfo =
-          new SubmitWorkInfo(taskSpec, fakeApplicationId, System.currentTimeMillis());
-      EventMetaData sourceMetaData =
-          new EventMetaData(EventMetaData.EventProducerConsumerType.INPUT, vertexName,
-              "NULL_VERTEX", null);
-      EventMetaData destinationMetaInfo = new TaskSpecBuilder().getDestingationMetaData(wx);
-
-      LOG.info("DBG: Number of splits: " + (eventList.size() - 1));
-      for (int i = 0; i < eventList.size() - 1; i++) {
-        // Creating the TezEvent here itself, since it's easy to serialize.
-        Event event = eventList.get(i + 1);
-        TaskLocationHint hint = hints.get(i);
-        Set<String> hosts = hint.getHosts();
-        LOG.info("DBG: Using locations: " + hosts.toString());
-        if (hosts.size() != 1) {
-          LOG.warn("DBG: Bad # of locations: " + hosts.size());
-        }
-        SplitLocationInfo[] locations = new SplitLocationInfo[hosts.size()];
-
-        int j = 0;
-        for (String host : hosts) {
-          locations[j++] = new SplitLocationInfo(host, false);
-        }
-        TezEvent tezEvent = new TezEvent(event, sourceMetaData, System.currentTimeMillis());
-        tezEvent.setDestinationInfo(destinationMetaInfo);
-
-        bos.reset();
-        dob.reset();
-        tezEvent.write(dob);
-
-        byte[] submitWorkBytes = SubmitWorkInfo.toBytes(submitWorkInfo);
-
-        result[i] =
-            new LlapInputSplit(i, submitWorkBytes, dob.getData(), locations, schema);
-      }
-      return result;
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-  }
-
-    /**
-   * Returns a local resource representing a jar. This resource will be used to execute the plan on
-   * the cluster.
-   *
-   * @param localJarPath
-   *          Local path to the jar to be localized.
-   * @return LocalResource corresponding to the localized hive exec resource.
-   * @throws IOException
-   *           when any file system related call fails.
-   * @throws LoginException
-   *           when we are unable to determine the user.
-   * @throws URISyntaxException
-   *           when current jar location cannot be determined.
-   */
-  private LocalResource createJarLocalResource(String localJarPath, DagUtils utils,
-      Configuration conf)
-    throws IOException, LoginException, IllegalArgumentException, FileNotFoundException {
-    FileStatus destDirStatus = utils.getHiveJarDirectory(conf);
-    assert destDirStatus != null;
-    Path destDirPath = destDirStatus.getPath();
-
-    Path localFile = new Path(localJarPath);
-    String sha = getSha(localFile, conf);
-
-    String destFileName = localFile.getName();
-
-    // Now, try to find the file based on SHA and name. Currently we require exact name match.
-    // We could also allow cutting off versions and other stuff provided that SHA matches...
-    destFileName = FilenameUtils.removeExtension(destFileName) + "-" + sha
-      + FilenameUtils.EXTENSION_SEPARATOR + FilenameUtils.getExtension(destFileName);
-
-    // TODO: if this method is ever called on more than one jar, getting the dir and the
-    // list need to be refactored out to be done only once.
-    Path destFile = new Path(destDirPath.toString() + "/" + destFileName);
-    return utils.localizeResource(localFile, destFile, LocalResourceType.FILE, conf);
-  }
-
-  private String getSha(Path localFile, Configuration conf)
-    throws IOException, IllegalArgumentException {
-    InputStream is = null;
-    try {
-      FileSystem localFs = FileSystem.getLocal(conf);
-      is = localFs.open(localFile);
-      return DigestUtils.sha256Hex(is);
-    } finally {
-      if (is != null) {
-        is.close();
-      }
-    }
-  }
-
-  @Override
-  public String getDisplayString(String[] children) {
-    assert children.length == 2;
-    return getStandardDisplayString("get_splits", children);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFExecuteSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFExecuteSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFExecuteSplits.java
new file mode 100644
index 0000000..12759ab
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFExecuteSplits.java
@@ -0,0 +1,124 @@
+/**
+ * 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.ql.udf.generic;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.UDFType;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDTFGetSplits.PlanFragment;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.RecordReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * GenericUDTFExecuteSplits.
+ *
+ */
+@Description(name = "execute_splits", value = "_FUNC_(string,int) - "
+    + "Returns an array of length int serialized splits for the referenced tables string.")
+@UDFType(deterministic = false)
+public class GenericUDTFExecuteSplits extends GenericUDTFGetSplits {
+
+  private static final Logger LOG = LoggerFactory.getLogger(GenericUDTFExecuteSplits.class);
+
+  @Override
+  public StructObjectInspector initialize(ObjectInspector[] arguments)
+    throws UDFArgumentException {
+
+    LOG.debug("initializing ExecuteSplits");
+
+    if (SessionState.get() == null || SessionState.get().getConf() == null) {
+      throw new IllegalStateException("Cannot run execute splits outside HS2");
+    }
+
+    if (arguments.length != 2) {
+      throw new UDFArgumentLengthException("The function execute_splits accepts 2 arguments.");
+    } else if (!(arguments[0] instanceof StringObjectInspector)) {
+      LOG.error("Got "+arguments[0].getTypeName()+" instead of string.");
+      throw new UDFArgumentTypeException(0, "\""
+          + "string\" is expected at function execute_splits, " + "but \""
+          + arguments[0].getTypeName() + "\" is found");
+    } else if (!(arguments[1] instanceof IntObjectInspector)) {
+      LOG.error("Got "+arguments[1].getTypeName()+" instead of int.");
+      throw new UDFArgumentTypeException(1, "\""
+          + "int\" is expected at function execute_splits, " + "but \""
+          + arguments[1].getTypeName() + "\" is found");
+    }
+
+    stringOI = (StringObjectInspector) arguments[0];
+    intOI = (IntObjectInspector) arguments[1];
+
+    List<String> names = Arrays.asList("split_num","value");
+    List<ObjectInspector> fieldOIs = Arrays.<ObjectInspector>asList(
+      PrimitiveObjectInspectorFactory.javaIntObjectInspector,
+      PrimitiveObjectInspectorFactory.javaStringObjectInspector);
+    StructObjectInspector outputOI = ObjectInspectorFactory.getStandardStructObjectInspector(names, fieldOIs);
+
+    LOG.debug("done initializing GenericUDTFExecuteSplits");
+    return outputOI;
+  }
+
+  @Override
+  public void process(Object[] arguments) throws HiveException {
+
+    String query = stringOI.getPrimitiveJavaObject(arguments[0]);
+    int num = intOI.get(arguments[1]);
+
+    PlanFragment fragment = createPlanFragment(query, num);
+    try {
+      InputFormat<NullWritable, Text> format = (InputFormat<NullWritable,Text>)(Class.forName("org.apache.hadoop.hive.llap.LlapInputFormat").newInstance());
+      int index = 0;
+      for (InputSplit s: getSplits(jc, num, fragment.work, fragment.schema)) {
+        RecordReader<NullWritable, Text> reader = format.getRecordReader(s,fragment.jc,null);
+        Text value = reader.createValue();
+        NullWritable key = reader.createKey();
+        index++;
+        while(reader.next(key,value)) {
+          Object[] os = new Object[2];
+          os[0] = index;
+          os[1] = value.toString();
+          forward(os);
+        }
+      }
+    } catch(Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
+  @Override
+  public void close() throws HiveException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
new file mode 100644
index 0000000..ebb0ca5
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
@@ -0,0 +1,420 @@
+/**
+ * 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.ql.udf.generic;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+
+import javax.security.auth.login.LoginException;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.llap.LlapInputSplit;
+import org.apache.hadoop.hive.llap.LlapOutputFormat;
+import org.apache.hadoop.hive.llap.SubmitWorkInfo;
+import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.exec.tez.DagUtils;
+import org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator;
+import org.apache.hadoop.hive.ql.exec.tez.TezTask;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.MapWork;
+import org.apache.hadoop.hive.ql.plan.TezWork;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.UDFType;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.SplitLocationInfo;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.TaskLocationHint;
+import org.apache.tez.dag.api.TaskSpecBuilder;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * GenericUDTFGetSplits.
+ *
+ */
+@Description(name = "get_splits", value = "_FUNC_(string,int) - "
+    + "Returns an array of length int serialized splits for the referenced tables string.")
+@UDFType(deterministic = false)
+public class GenericUDTFGetSplits extends GenericUDTF {
+
+  private static final Logger LOG = LoggerFactory.getLogger(GenericUDTFGetSplits.class);
+
+  private static final String LLAP_INTERNAL_INPUT_FORMAT_NAME = "org.apache.hadoop.hive.llap.LlapInputFormat";
+
+  protected transient StringObjectInspector stringOI;
+  protected transient IntObjectInspector intOI;
+  protected transient JobConf jc;
+  private ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
+  private DataOutput dos = new DataOutputStream(bos);
+
+  @Override
+  public StructObjectInspector initialize(ObjectInspector[] arguments)
+    throws UDFArgumentException {
+
+    LOG.debug("initializing GenericUDFGetSplits");
+
+    if (SessionState.get() == null || SessionState.get().getConf() == null) {
+      throw new IllegalStateException("Cannot run get splits outside HS2");
+    }
+
+    LOG.debug("Initialized conf, jc and metastore connection");
+
+    if (arguments.length != 2) {
+      throw new UDFArgumentLengthException("The function GET_SPLITS accepts 2 arguments.");
+    } else if (!(arguments[0] instanceof StringObjectInspector)) {
+      LOG.error("Got "+arguments[0].getTypeName()+" instead of string.");
+      throw new UDFArgumentTypeException(0, "\""
+          + "string\" is expected at function GET_SPLITS, " + "but \""
+          + arguments[0].getTypeName() + "\" is found");
+    } else if (!(arguments[1] instanceof IntObjectInspector)) {
+      LOG.error("Got "+arguments[1].getTypeName()+" instead of int.");
+      throw new UDFArgumentTypeException(1, "\""
+          + "int\" is expected at function GET_SPLITS, " + "but \""
+          + arguments[1].getTypeName() + "\" is found");
+    }
+
+    stringOI = (StringObjectInspector) arguments[0];
+    intOI = (IntObjectInspector) arguments[1];
+
+    List<String> names = Arrays.asList("if_class","split_class","split");
+    List<ObjectInspector> fieldOIs = Arrays.<ObjectInspector>asList(
+      PrimitiveObjectInspectorFactory.javaStringObjectInspector,
+      PrimitiveObjectInspectorFactory.javaStringObjectInspector,
+      PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector);
+    StructObjectInspector outputOI = ObjectInspectorFactory.getStandardStructObjectInspector(names, fieldOIs);
+
+    LOG.debug("done initializing GenericUDFGetSplits");
+    return outputOI;
+  }
+
+  public static class PlanFragment {
+    public JobConf jc;
+    public TezWork work;
+    public Schema schema;
+
+    public PlanFragment(TezWork work, Schema schema, JobConf jc) {
+      this.work = work;
+      this.schema = schema;
+      this.jc = jc;
+    }
+  }
+
+  @Override
+  public void process(Object[] arguments) throws HiveException {
+
+    String query = stringOI.getPrimitiveJavaObject(arguments[0]);
+    int num = intOI.get(arguments[1]);
+
+    PlanFragment fragment = createPlanFragment(query, num);
+    TezWork tezWork = fragment.work;
+    Schema schema = fragment.schema;
+
+    try {
+      for (InputSplit s: getSplits(jc, num, tezWork, schema)) {
+        Object[] os = new Object[3];
+        os[0] = LLAP_INTERNAL_INPUT_FORMAT_NAME;
+        os[1] = s.getClass().getName();
+        bos.reset();
+        s.write(dos);
+        byte[] frozen = bos.toByteArray();
+        os[2] = frozen;
+        forward(os);
+      }
+    } catch(Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
+  public PlanFragment createPlanFragment(String query, int num) throws HiveException {
+
+    HiveConf conf = new HiveConf(SessionState.get().getConf());
+    HiveConf.setVar(conf, ConfVars.HIVEFETCHTASKCONVERSION, "none");
+    HiveConf.setVar(conf, HiveConf.ConfVars.HIVEQUERYRESULTFILEFORMAT, "Llap");
+
+    String originalMode = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_MODE);
+    HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_MODE, "llap");
+    HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_TEZ_GENERATE_CONSISTENT_SPLITS, true);
+    HiveConf.setBoolVar(conf, HiveConf.ConfVars.LLAP_CLIENT_CONSISTENT_SPLITS, true);
+
+    try {
+      jc = DagUtils.getInstance().createConfiguration(conf);
+    } catch (IOException e) {
+      throw new HiveException(e);
+    }
+
+    Driver driver = new Driver(conf);
+    CommandProcessorResponse cpr;
+
+    LOG.info("setting fetch.task.conversion to none and query file format to \""
+        + LlapOutputFormat.class.getName()+"\"");
+
+    cpr = driver.compileAndRespond(query);
+    if(cpr.getResponseCode() != 0) {
+      throw new HiveException("Failed to compile query: "+cpr.getException());
+    }
+
+    QueryPlan plan = driver.getPlan();
+    List<Task<?>> roots = plan.getRootTasks();
+    Schema schema = plan.getResultSchema();
+
+    if (roots == null || roots.size() != 1 || !(roots.get(0) instanceof TezTask)) {
+      throw new HiveException("Was expecting a single TezTask.");
+    }
+
+    TezWork tezWork = ((TezTask)roots.get(0)).getWork();
+
+    if (tezWork.getAllWork().size() != 1) {
+
+      String tableName = "table_"+UUID.randomUUID().toString().replaceAll("[^A-Za-z0-9 ]", "");
+
+      String ctas = "create temporary table "+tableName+" as "+query;
+      LOG.info("CTAS: "+ctas);
+
+      try {
+        HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_MODE, originalMode);
+        cpr = driver.run(ctas, false);
+      } catch(CommandNeedRetryException e) {
+        throw new HiveException(e);
+      }
+
+      if(cpr.getResponseCode() != 0) {
+        throw new HiveException("Failed to create temp table: " + cpr.getException());
+      }
+
+      HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_MODE, "llap");
+      query = "select * from " + tableName;
+      cpr = driver.compileAndRespond(query);
+      if(cpr.getResponseCode() != 0) {
+        throw new HiveException("Failed to create temp table: "+cpr.getException());
+      }
+
+      plan = driver.getPlan();
+      roots = plan.getRootTasks();
+      schema = plan.getResultSchema();
+
+      if (roots == null || roots.size() != 1 || !(roots.get(0) instanceof TezTask)) {
+        throw new HiveException("Was expecting a single TezTask.");
+      }
+
+      tezWork = ((TezTask)roots.get(0)).getWork();
+    }
+
+    return new PlanFragment(tezWork, schema, jc);
+  }
+
+  public InputSplit[] getSplits(JobConf job, int numSplits, TezWork work, Schema schema)
+    throws IOException {
+
+    DAG dag = DAG.create(work.getName());
+    dag.setCredentials(job.getCredentials());
+    // TODO: set access control? TezTask.setAccessControlsForCurrentUser(dag);
+
+    DagUtils utils = DagUtils.getInstance();
+    Context ctx = new Context(job);
+    MapWork mapWork = (MapWork) work.getAllWork().get(0);
+    // bunch of things get setup in the context based on conf but we need only the MR tmp directory
+    // for the following method.
+    JobConf wxConf = utils.initializeVertexConf(job, ctx, mapWork);
+    Path scratchDir = utils.createTezDir(ctx.getMRScratchDir(), job);
+    FileSystem fs = scratchDir.getFileSystem(job);
+    try {
+      LocalResource appJarLr = createJarLocalResource(utils.getExecJarPathLocal(), utils, job);
+      Vertex wx = utils.createVertex(wxConf, mapWork, scratchDir, appJarLr,
+          new ArrayList<LocalResource>(), fs, ctx, false, work,
+          work.getVertexType(mapWork));
+      String vertexName = wx.getName();
+      dag.addVertex(wx);
+      utils.addCredentials(mapWork, dag);
+
+
+      // we have the dag now proceed to get the splits:
+      HiveSplitGenerator splitGenerator = new HiveSplitGenerator(null);
+      Preconditions.checkState(HiveConf.getBoolVar(wxConf,
+              HiveConf.ConfVars.HIVE_TEZ_GENERATE_CONSISTENT_SPLITS));
+      Preconditions.checkState(HiveConf.getBoolVar(wxConf,
+              HiveConf.ConfVars.LLAP_CLIENT_CONSISTENT_SPLITS));
+      splitGenerator.initializeSplitGenerator(wxConf, mapWork);
+      List<Event> eventList = splitGenerator.initialize();
+
+      // hack - just serializing with kryo for now. This needs to be done properly
+      InputSplit[] result = new InputSplit[eventList.size() - 1];
+      DataOutputBuffer dob = new DataOutputBuffer();
+
+      InputConfigureVertexTasksEvent configureEvent
+        = (InputConfigureVertexTasksEvent) eventList.get(0);
+
+      List<TaskLocationHint> hints = configureEvent.getLocationHint().getTaskLocationHints();
+
+      Preconditions.checkState(hints.size() == eventList.size() - 1);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("NumEvents=" + eventList.size());
+        LOG.debug("NumSplits=" + result.length);
+      }
+
+      ApplicationId fakeApplicationId
+        = ApplicationId.newInstance(Math.abs(new Random().nextInt()), 0);
+
+      LOG.info("Number of splits: " + (eventList.size() - 1));
+      for (int i = 0; i < eventList.size() - 1; i++) {
+
+        TaskSpec taskSpec =
+          new TaskSpecBuilder().constructTaskSpec(dag, vertexName,
+              eventList.size() - 1, fakeApplicationId, i);
+
+        SubmitWorkInfo submitWorkInfo =
+          new SubmitWorkInfo(taskSpec, fakeApplicationId, System.currentTimeMillis());
+        EventMetaData sourceMetaData =
+          new EventMetaData(EventMetaData.EventProducerConsumerType.INPUT, vertexName,
+              "NULL_VERTEX", null);
+        EventMetaData destinationMetaInfo = new TaskSpecBuilder().getDestingationMetaData(wx);
+
+        // Creating the TezEvent here itself, since it's easy to serialize.
+        Event event = eventList.get(i + 1);
+        TaskLocationHint hint = hints.get(i);
+        Set<String> hosts = hint.getHosts();
+        if (hosts.size() != 1) {
+          LOG.warn("Bad # of locations: " + hosts.size());
+        }
+        SplitLocationInfo[] locations = new SplitLocationInfo[hosts.size()];
+
+        int j = 0;
+        for (String host : hosts) {
+          locations[j++] = new SplitLocationInfo(host, false);
+        }
+        TezEvent tezEvent = new TezEvent(event, sourceMetaData, System.currentTimeMillis());
+        tezEvent.setDestinationInfo(destinationMetaInfo);
+
+        bos.reset();
+        dob.reset();
+        tezEvent.write(dob);
+
+        byte[] submitWorkBytes = SubmitWorkInfo.toBytes(submitWorkInfo);
+
+        result[i] = new LlapInputSplit(i, submitWorkBytes, dob.getData(), locations, schema);
+      }
+      return result;
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Returns a local resource representing a jar. This resource will be used to execute the plan on
+   * the cluster.
+   *
+   * @param localJarPath
+   *          Local path to the jar to be localized.
+   * @return LocalResource corresponding to the localized hive exec resource.
+   * @throws IOException
+   *           when any file system related call fails.
+   * @throws LoginException
+   *           when we are unable to determine the user.
+   * @throws URISyntaxException
+   *           when current jar location cannot be determined.
+   */
+  private LocalResource createJarLocalResource(String localJarPath, DagUtils utils,
+      Configuration conf)
+    throws IOException, LoginException, IllegalArgumentException, FileNotFoundException {
+    FileStatus destDirStatus = utils.getHiveJarDirectory(conf);
+    assert destDirStatus != null;
+    Path destDirPath = destDirStatus.getPath();
+
+    Path localFile = new Path(localJarPath);
+    String sha = getSha(localFile, conf);
+
+    String destFileName = localFile.getName();
+
+    // Now, try to find the file based on SHA and name. Currently we require exact name match.
+    // We could also allow cutting off versions and other stuff provided that SHA matches...
+    destFileName = FilenameUtils.removeExtension(destFileName) + "-" + sha
+      + FilenameUtils.EXTENSION_SEPARATOR + FilenameUtils.getExtension(destFileName);
+
+    // TODO: if this method is ever called on more than one jar, getting the dir and the
+    // list need to be refactored out to be done only once.
+    Path destFile = new Path(destDirPath.toString() + "/" + destFileName);
+    return utils.localizeResource(localFile, destFile, LocalResourceType.FILE, conf);
+  }
+
+  private String getSha(Path localFile, Configuration conf)
+    throws IOException, IllegalArgumentException {
+    InputStream is = null;
+    try {
+      FileSystem localFs = FileSystem.getLocal(conf);
+      is = localFs.open(localFile);
+      return DigestUtils.sha256Hex(is);
+    } finally {
+      if (is != null) {
+        is.close();
+      }
+    }
+  }
+
+  @Override
+  public void close() throws HiveException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java b/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
index 5cabb6a..5db8c48 100644
--- a/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
+++ b/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
@@ -17,7 +17,7 @@ import org.apache.tez.runtime.api.impl.TaskSpec;
 // Proxy class within the tez.api package to access package private methods.
 public class TaskSpecBuilder {
 
-  public TaskSpec constructTaskSpec(DAG dag, String vertexName, int numSplits, ApplicationId appId) {
+  public TaskSpec constructTaskSpec(DAG dag, String vertexName, int numSplits, ApplicationId appId, int index) {
     Vertex vertex = dag.getVertex(vertexName);
     ProcessorDescriptor processorDescriptor = vertex.getProcessorDescriptor();
     List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> inputs =
@@ -43,7 +43,7 @@ public class TaskSpecBuilder {
 
     TezDAGID dagId = TezDAGID.getInstance(appId, 0);
     TezVertexID vertexId = TezVertexID.getInstance(dagId, 0);
-    TezTaskID taskId = TezTaskID.getInstance(vertexId, 0);
+    TezTaskID taskId = TezTaskID.getInstance(vertexId, index);
     TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 0);
     return new TaskSpec(taskAttemptId, dag.getName(), vertexName, numSplits, processorDescriptor, inputSpecs, outputSpecs, null);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
index c49231c..7b516fe 100644
--- a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
@@ -79,46 +79,52 @@ public class TestLlapOutputFormat {
   @Test
   public void testValues() throws Exception {
     JobConf job = new JobConf();
-    job.set(LlapOutputFormat.LLAP_OF_ID_KEY, "foobar");
-    LlapOutputFormat format = new LlapOutputFormat();
 
-    HiveConf conf = new HiveConf();
-    Socket socket = new Socket("localhost",
-        conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT));
+    for (int k = 0; k < 5; ++k) {
+      String id = "foobar"+k;
+      job.set(LlapOutputFormat.LLAP_OF_ID_KEY, id);
+      LlapOutputFormat format = new LlapOutputFormat();
 
-    LOG.debug("Socket connected");
+      HiveConf conf = new HiveConf();
+      Socket socket = new Socket("localhost",
+          conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT));
 
-    socket.getOutputStream().write("foobar".getBytes());
-    socket.getOutputStream().write(0);
-    socket.getOutputStream().flush();
+      LOG.debug("Socket connected");
 
-    Thread.sleep(3000);
+      socket.getOutputStream().write(id.getBytes());
+      socket.getOutputStream().write(0);
+      socket.getOutputStream().flush();
 
-    LOG.debug("Data written");
+      Thread.sleep(3000);
 
-    RecordWriter<NullWritable, Text> writer = format.getRecordWriter(null, job, null, null);
-    Text text = new Text();
+      LOG.debug("Data written");
 
-    LOG.debug("Have record writer");
+      RecordWriter<NullWritable, Text> writer = format.getRecordWriter(null, job, null, null);
+      Text text = new Text();
 
-    for (int i = 0; i < 10; ++i) {
-      text.set(""+i);
-      writer.write(NullWritable.get(),text);
-    }
+      LOG.debug("Have record writer");
 
-    writer.close(null);
+      for (int i = 0; i < 10; ++i) {
+        text.set(""+i);
+        writer.write(NullWritable.get(),text);
+      }
 
-    InputStream in = socket.getInputStream();
-    RecordReader reader = new LlapRecordReader(in, null, Text.class);
+      writer.close(null);
 
-    LOG.debug("Have record reader");
+      InputStream in = socket.getInputStream();
+      RecordReader reader = new LlapRecordReader(in, null, Text.class);
 
-    int count = 0;
-    while(reader.next(NullWritable.get(), text)) {
-      LOG.debug(text.toString());
-      count++;
-    }
+      LOG.debug("Have record reader");
+
+      int count = 0;
+      while(reader.next(NullWritable.get(), text)) {
+        LOG.debug(text.toString());
+        count++;
+      }
 
-    Assert.assertEquals(count,10);
+      reader.close();
+
+      Assert.assertEquals(count,10);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/test/queries/clientpositive/udf_get_splits.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udf_get_splits.q b/ql/src/test/queries/clientpositive/udf_get_splits.q
deleted file mode 100644
index 70400e8..0000000
--- a/ql/src/test/queries/clientpositive/udf_get_splits.q
+++ /dev/null
@@ -1,6 +0,0 @@
-set hive.fetch.task.conversion=more;
-
-DESCRIBE FUNCTION get_splits;
-DESCRIBE FUNCTION EXTENDED get_splits;
-
-select r.if_class as ic, r.split_class as sc, hash(r.split) as h, length(r.split) as l from (select explode(get_splits("select key, count(*) from src where key % 2 = 0 group by key",5)) as r) t;

http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/test/queries/clientpositive/udtf_get_splits.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udtf_get_splits.q b/ql/src/test/queries/clientpositive/udtf_get_splits.q
new file mode 100644
index 0000000..f378dca
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/udtf_get_splits.q
@@ -0,0 +1,43 @@
+set hive.fetch.task.conversion=more;
+set hive.mapred.mode=nonstrict;
+set mapred.max.split.size=100;
+set mapred.min.split.size.per.node=100;
+set mapred.min.split.size.per.rack=100;
+set mapred.max.split.size=100;
+set tez.grouping.max-size=100;
+set tez.grouping.min-size=100;
+
+DESCRIBE FUNCTION get_splits;
+DESCRIBE FUNCTION execute_splits;
+
+select r1, r2, floor(length(r3)/100000)
+from
+  (select
+    get_splits(
+      "select key, count(*) from srcpart where key % 2 = 0 group by key",
+      5) as (r1, r2, r3)) t;
+
+select r1, r2, floor(length(r3)/100000)
+from
+  (select
+    get_splits(
+      "select key from srcpart where key % 2 = 0",
+      5) as (r1, r2, r3)) t;
+
+show tables;
+
+select r1, r2
+from
+  (select
+    execute_splits(
+      "select key from srcpart where key % 2 = 0",
+      1) as (r1, r2)) t;
+
+select r1, r2
+from
+  (select
+    execute_splits(
+      "select key from srcpart where key % 2 = 0",
+      5) as (r1, r2)) t;
+
+select count(*) from (select key from srcpart where key % 2 = 0) t;


[09/39] hive git commit: HIVE-13202: LLAP: Replace use of ServerSocket with netty in LlapOutputFormatService

Posted by jd...@apache.org.
HIVE-13202: LLAP: Replace use of ServerSocket with netty in LlapOutputFormatService


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/81b26df9
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/81b26df9
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/81b26df9

Branch: refs/heads/master
Commit: 81b26df9ed00e9db671c57aece8e51bf62365e34
Parents: d8a9531
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu Mar 3 12:57:43 2016 -0800
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu Mar 3 12:57:43 2016 -0800

----------------------------------------------------------------------
 .../apache/hive/jdbc/TestJdbcWithMiniLlap.java  |  74 +++++++--
 .../hadoop/hive/llap/ChannelOutputStream.java   | 141 +++++++++++++++++
 .../hive/llap/LlapOutputFormatService.java      | 155 ++++++++++++-------
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |  12 +-
 4 files changed, 308 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/81b26df9/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
index 88e2e55..98daab4 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
@@ -37,6 +37,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.ExecutionException;
@@ -86,6 +88,7 @@ public class TestJdbcWithMiniLlap {
   private static Path kvDataFilePath;
   private static final String tmpDir = System.getProperty("test.tmp.dir");
 
+  private static HiveConf conf = null;
   private Connection hs2Conn = null;
 
   @BeforeClass
@@ -98,7 +101,7 @@ public class TestJdbcWithMiniLlap {
       System.out.println("Setting hive-site: "+HiveConf.getHiveSiteLocation());
     }
 
-    HiveConf conf = new HiveConf();
+    conf = new HiveConf();
     conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
     // Necessary for GetSplits()/LlapInputFormat,
     // the config generated for the query fragment needs to include the MapWork
@@ -109,7 +112,7 @@ public class TestJdbcWithMiniLlap {
     conf.addResource(new URL("file://" + new File(confDir).toURI().getPath()
         + "/llap-daemon-site.xml"));
 
-    miniHS2 = new MiniHS2(conf, MiniClusterType.LLAP, true);
+    miniHS2 = new MiniHS2(conf, MiniClusterType.LLAP);
 
     dataFileDir = conf.get("test.data.files").replace('\\', '/').replace("c:", "");
     kvDataFilePath = new Path(dataFileDir, "kv1.txt");
@@ -160,21 +163,54 @@ public class TestJdbcWithMiniLlap {
     stmt.close();
   }
 
-  @Test
-  public void testLlapInputFormatEndToEnd() throws Exception {
-    createTestTable("testtab1");
+  private static boolean timedOut = false;
+
+  private static class TestTimerTask extends TimerTask {
+    private boolean timedOut = false;
+    private Thread threadToInterrupt;
+
+    public TestTimerTask(Thread threadToInterrupt) {
+      this.threadToInterrupt = threadToInterrupt;
+    }
+
+    @Override
+    public void run() {
+      System.out.println("Test timed out!");
+      timedOut = true;
+      threadToInterrupt.interrupt();
+    }
+
+    public boolean isTimedOut() {
+      return timedOut;
+    }
+
+    public void setTimedOut(boolean timedOut) {
+      this.timedOut = timedOut;
+    }
+
+  }
+
+  private int getLlapIFRowCount(String query, int numSplits) throws Exception {
+    // Add a timer task to stop this test if it has not finished in a reasonable amount of time.
+    Timer timer = new Timer();
+    long delay = 30000;
+    TestTimerTask timerTask = new TestTimerTask(Thread.currentThread());
+    timer.schedule(timerTask, delay);
+
+    // Setup LlapInputFormat
     String url = miniHS2.getJdbcURL();
     String user = System.getProperty("user.name");
     String pwd = user;
-    String query = "select * from testtab1 where under_col = 0";
 
     LlapInputFormat inputFormat = new LlapInputFormat(url, user, pwd, query);
-    JobConf job = new JobConf();
-    int numSplits = 1;
+
+    // Get splits
+    JobConf job = new JobConf(conf);
 
     InputSplit[] splits = inputFormat.getSplits(job, numSplits);
-    assert(splits.length > 0);
+    assertTrue(splits.length > 0);
 
+    // Fetch rows from splits
     boolean first = true;
     int rowCount = 0;
     for (InputSplit split : splits) {
@@ -198,6 +234,26 @@ public class TestJdbcWithMiniLlap {
         ++rowCount;
       }
     }
+
+    timer.cancel();
+    assertFalse("Test timed out", timerTask.isTimedOut());
+
+    return rowCount;
+  }
+
+  @Test
+  public void testLlapInputFormatEndToEnd() throws Exception {
+    createTestTable("testtab1");
+
+    int rowCount;
+
+    String query = "select * from testtab1 where under_col = 0";
+    rowCount = getLlapIFRowCount(query, 1);
     assertEquals(3, rowCount);
+
+    // Try empty rows query
+    query = "select * from testtab1 where true = false";
+    rowCount = getLlapIFRowCount(query, 1);
+    assertEquals(0, rowCount);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/81b26df9/ql/src/java/org/apache/hadoop/hive/llap/ChannelOutputStream.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/ChannelOutputStream.java b/ql/src/java/org/apache/hadoop/hive/llap/ChannelOutputStream.java
new file mode 100644
index 0000000..e861791
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/llap/ChannelOutputStream.java
@@ -0,0 +1,141 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * OutputStream to write to the Netty Channel
+ */
+public class ChannelOutputStream extends OutputStream {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ChannelOutputStream.class);
+
+  private ChannelHandlerContext chc;
+  private int bufSize;
+  private String id;
+  private ByteBuf buf;
+  private byte[] singleByte = new byte[1];
+  private boolean closed = false;
+
+  private ChannelFutureListener listener = new ChannelFutureListener() {
+    @Override
+    public void operationComplete(ChannelFuture future) {
+      if (future.isCancelled()) {
+        LOG.error(id + " was cancelled");
+      } else if (!future.isSuccess()) {
+        LOG.error("Error on ID " + id, future.cause());
+      }
+    }
+  };
+
+  public ChannelOutputStream(ChannelHandlerContext chc, String id, int bufSize) {
+    this.chc = chc;
+    this.id = id;
+    this.bufSize = bufSize;
+    this.buf = chc.alloc().buffer(bufSize);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    singleByte[0] = (byte) b;
+    write(singleByte, 0, 1);
+  }
+
+  @Override
+  public void write(byte[] b) throws IOException {
+    write(b, 0, b.length);
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    int currentOffset = off;
+    int bytesRemaining = len;
+
+    while (bytesRemaining + buf.readableBytes() > bufSize) {
+      int iterationLen = bufSize - buf.readableBytes();
+      writeInternal(b, currentOffset, iterationLen);
+      currentOffset += iterationLen;
+      bytesRemaining -= iterationLen;
+    }
+
+    if (bytesRemaining > 0) {
+      writeInternal(b, currentOffset, bytesRemaining);
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (buf.isReadable()) {
+      writeToChannel();
+    }
+    chc.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closed) {
+      throw new IOException("Already closed: " + id);
+    }
+
+    try {
+      flush();
+    } catch (IOException err) {
+      LOG.error("Error flushing stream before close", err);
+    }
+
+    try {
+      chc.close().addListener(listener).sync();
+    } catch (InterruptedException err) {
+      throw new IOException(err);
+    } finally {
+      buf.release();
+      buf = null;
+      chc = null;
+      closed = true;
+    }
+  }
+
+  private void writeToChannel() throws IOException {
+    if (closed) {
+      throw new IOException("Already closed: " + id);
+    }
+
+    chc.write(buf.copy()).addListener(listener);
+    buf.clear();
+  }
+
+  private void writeInternal(byte[] b, int off, int len) throws IOException {
+    if (closed) {
+      throw new IOException("Already closed: " + id);
+    }
+
+    buf.writeBytes(b, off, len);
+    if (buf.readableBytes() >= bufSize) {
+      writeToChannel();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/81b26df9/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
index a197d7b..b39f085 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
@@ -19,9 +19,6 @@ package org.apache.hadoop.hive.llap;
 import java.util.Map;
 import java.util.HashMap;
 import java.io.IOException;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.io.InputStream;
 import java.io.OutputStream;
 
 import org.slf4j.Logger;
@@ -45,8 +42,22 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.SimpleChannelInboundHandler;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.DelimiterBasedFrameDecoder;
+import io.netty.handler.codec.Delimiters;
+import io.netty.handler.codec.string.StringDecoder;
+import io.netty.handler.codec.string.StringEncoder;
+import io.netty.util.concurrent.Future;
+
 
 /**
  *
@@ -57,18 +68,17 @@ public class LlapOutputFormatService {
 
   private static LlapOutputFormatService service;
   private final Map<String, RecordWriter> writers;
-  private final ServerSocket socket;
   private final HiveConf conf;
-  private final ExecutorService executor;
   private static final int WAIT_TIME = 5;
+  private static final int MAX_QUERY_ID_LENGTH = 256;
+
+  private EventLoopGroup eventLoopGroup;
+  private ServerBootstrap serverBootstrap;
+  private ChannelFuture listeningChannelFuture;
 
   private LlapOutputFormatService() throws IOException {
     writers = new HashMap<String, RecordWriter>();
     conf = new HiveConf();
-    executor = Executors.newSingleThreadExecutor(
-      new ThreadFactoryBuilder().setDaemon(true).setNameFormat("LLAP output %d").build());
-    socket = new ServerSocket(
-      conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT));
   }
 
   public static LlapOutputFormatService get() throws IOException {
@@ -80,52 +90,34 @@ public class LlapOutputFormatService {
   }
 
   public void start() throws IOException {
-    executor.submit(new Runnable() {
-        byte[] buffer = new byte[4096];
-        @Override
-        public void run() {
-          while (true) {
-            Socket s = null;
-            try {
-              s = socket.accept();
-              String id = readId(s);
-              LOG.debug("Received: "+id);
-              registerReader(s, id);
-            } catch (IOException io) {
-              if (s != null) {
-                try{
-                  s.close();
-                } catch (IOException io2) {
-                  // ignore
-                }
-              }
-            }
-          }
-        }
-
-        private String readId(Socket s) throws IOException {
-          InputStream in = s.getInputStream();
-          int idx = 0;
-          while((buffer[idx++] = (byte)in.read()) != '\0') {}
-          return new String(buffer,0,idx-1);
-        }
-
-        private void registerReader(Socket s, String id) throws IOException {
-          synchronized(service) {
-            LOG.debug("registering socket for: "+id);
-            LlapRecordWriter writer = new LlapRecordWriter(s.getOutputStream());
-            writers.put(id, writer);
-            service.notifyAll();
-          }
-        }
-      }
-      );
+    LOG.info("Starting LlapOutputFormatService");
+
+    int port = conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
+    eventLoopGroup = new NioEventLoopGroup(1);
+    serverBootstrap = new ServerBootstrap();
+    serverBootstrap.group(eventLoopGroup);
+    serverBootstrap.channel(NioServerSocketChannel.class);
+    serverBootstrap.childHandler(new LlapOutputFormatServiceChannelHandler());
+    try {
+      LOG.info("LlapOutputFormatService: Binding to port " + port);
+      listeningChannelFuture = serverBootstrap.bind(port).sync();
+    } catch (InterruptedException err) {
+      throw new IOException("LlapOutputFormatService: Error binding to port " + port, err);
+    }
   }
 
   public void stop() throws IOException, InterruptedException {
-    executor.shutdown();
-    executor.awaitTermination(WAIT_TIME, TimeUnit.SECONDS);
-    socket.close();
+    LOG.info("Stopping LlapOutputFormatService");
+
+    if (listeningChannelFuture != null) {
+      listeningChannelFuture.channel().close().sync();
+      listeningChannelFuture = null;
+    } else {
+      LOG.warn("LlapOutputFormatService does not appear to have a listening port to close.");
+    }
+
+    Future terminationFuture = eventLoopGroup.shutdownGracefully(1, WAIT_TIME, TimeUnit.SECONDS);
+    terminationFuture.sync();
   }
 
   public <K,V> RecordWriter<K, V> getWriter(String id) throws IOException, InterruptedException {
@@ -139,4 +131,59 @@ public class LlapOutputFormatService {
     LOG.info("Returning writer for: "+id);
     return writer;
   }
+
+  protected class LlapOutputFormatServiceHandler extends SimpleChannelInboundHandler<String> {
+    @Override
+    public void channelRead0(ChannelHandlerContext ctx, String msg) {
+      String id = msg;
+      registerReader(ctx, id);
+    }
+
+    private void registerReader(ChannelHandlerContext ctx, String id) {
+      synchronized(service) {
+        LOG.debug("registering socket for: "+id);
+        int bufSize = 128 * 1024; // configable?
+        OutputStream stream = new ChannelOutputStream(ctx, id, bufSize);
+        LlapRecordWriter writer = new LlapRecordWriter(stream);
+        writers.put(id, writer);
+
+        // Add listener to handle any cleanup for when the connection is closed
+        ctx.channel().closeFuture().addListener(new LlapOutputFormatChannelCloseListener(id));
+
+        service.notifyAll();
+      }
+    }
+  }
+
+  protected class LlapOutputFormatChannelCloseListener implements ChannelFutureListener {
+    private String id;
+
+    LlapOutputFormatChannelCloseListener(String id) {
+      this.id = id;
+    }
+
+    @Override
+    public void operationComplete(ChannelFuture future) throws Exception {
+      RecordWriter writer = null;
+
+      synchronized (service) {
+        writer = writers.remove(id);
+      }
+
+      if (writer == null) {
+        LOG.warn("Did not find a writer for ID " + id);
+      }
+    }
+  }
+
+  protected class LlapOutputFormatServiceChannelHandler extends ChannelInitializer<SocketChannel> {
+    @Override
+    public void initChannel(SocketChannel ch) throws Exception {
+        ch.pipeline().addLast(
+            new DelimiterBasedFrameDecoder(MAX_QUERY_ID_LENGTH, Delimiters.nulDelimiter()),
+            new StringDecoder(),
+            new StringEncoder(),
+            new LlapOutputFormatServiceHandler());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/81b26df9/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index 02439be..17f3895 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@ -203,17 +203,7 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
           }
         }
       }
-      try {
-        if ("org.apache.hadoop.hive.llap.LlapStorageHandler".equals(getConf().getTableInfo().getProperties().
-            get(hive_metastoreConstants.META_TABLE_STORAGE))) {
-          (new LlapOutputFormat())
-              .getRecordWriter(null,
-                  hconf instanceof JobConf ? (JobConf) hconf : new JobConf(hconf), null, null)
-              .close(null);
-        }
-      } catch (IOException e) {
-        // ignored
-      }
+
       try {
         for (int i = 0; i < updaters.length; i++) {
           if (updaters[i] != null) {


[32/39] hive git commit: Merge branch 'master' into llap

Posted by jd...@apache.org.
Merge branch 'master' into llap


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/bc75d72b
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bc75d72b
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bc75d72b

Branch: refs/heads/master
Commit: bc75d72b8e5ed8a538cf69d3b0a7556150e4713e
Parents: e073cce b621827
Author: Jason Dere <jd...@hortonworks.com>
Authored: Tue May 3 13:30:39 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Tue May 3 13:30:39 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/common/FileUtils.java    |   19 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    2 +-
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |   43 +-
 .../cli/session/TestHiveSessionImpl.java        |    2 +-
 .../test/resources/testconfiguration.properties |    4 +
 .../org/apache/hive/jdbc/HiveStatement.java     |   20 +-
 .../llap/tezplugins/LlapTaskCommunicator.java   |   47 +
 .../hive/metastore/HiveMetaStoreFsImpl.java     |   11 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   27 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   73 +-
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |    7 +-
 .../hive/ql/exec/vector/VectorExtractRow.java   |   12 +-
 .../ql/exec/vector/VectorizationContext.java    |    2 +-
 .../hadoop/hive/ql/history/HiveHistoryImpl.java |    8 +-
 .../ql/io/orc/ConvertTreeReaderFactory.java     | 3750 +++++++++++++
 .../hadoop/hive/ql/io/orc/SchemaEvolution.java  |   29 +-
 .../hive/ql/io/orc/TreeReaderFactory.java       |   86 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   27 +-
 .../hadoop/hive/ql/session/OperationLog.java    |    8 +-
 .../hadoop/hive/ql/session/SessionState.java    |    2 +-
 .../clientnegative/orc_replace_columns2.q       |    5 +-
 .../clientnegative/orc_replace_columns2_acid.q  |    5 +-
 .../clientnegative/orc_replace_columns3.q       |    3 +
 .../clientnegative/orc_replace_columns3_acid.q  |    3 +
 .../clientnegative/orc_type_promotion1.q        |    7 +-
 .../clientnegative/orc_type_promotion1_acid.q   |    7 +-
 .../clientnegative/orc_type_promotion2.q        |    5 +-
 .../clientnegative/orc_type_promotion2_acid.q   |    5 +-
 .../clientnegative/orc_type_promotion3.q        |    5 +-
 .../clientnegative/orc_type_promotion3_acid.q   |    5 +-
 .../clientpositive/orc_int_type_promotion.q     |    4 +
 .../clientpositive/orc_schema_evolution.q       |    2 +
 .../schema_evol_orc_acid_mapwork_part.q         |  846 ++-
 .../schema_evol_orc_acid_mapwork_table.q        |  804 ++-
 .../schema_evol_orc_acidvec_mapwork_part.q      |  843 ++-
 .../schema_evol_orc_acidvec_mapwork_table.q     |  801 ++-
 .../schema_evol_orc_nonvec_fetchwork_part.q     |  831 ++-
 .../schema_evol_orc_nonvec_fetchwork_table.q    |  824 ++-
 .../schema_evol_orc_nonvec_mapwork_part.q       |  833 ++-
 ...a_evol_orc_nonvec_mapwork_part_all_complex.q |  162 +
 ...evol_orc_nonvec_mapwork_part_all_primitive.q |  481 ++
 .../schema_evol_orc_nonvec_mapwork_table.q      |  824 ++-
 .../schema_evol_orc_vec_mapwork_part.q          |  831 ++-
 ...hema_evol_orc_vec_mapwork_part_all_complex.q |  162 +
 ...ma_evol_orc_vec_mapwork_part_all_primitive.q |  481 ++
 .../schema_evol_orc_vec_mapwork_table.q         |  819 ++-
 .../schema_evol_text_nonvec_mapwork_part.q      |    5 +-
 ..._evol_text_nonvec_mapwork_part_all_complex.q |    5 +-
 ...vol_text_nonvec_mapwork_part_all_primitive.q |    5 +-
 .../schema_evol_text_nonvec_mapwork_table.q     |    5 +-
 .../schema_evol_text_vec_mapwork_part.q         |    2 +-
 ...ema_evol_text_vec_mapwork_part_all_complex.q |    2 +-
 ...a_evol_text_vec_mapwork_part_all_primitive.q |    2 +-
 .../schema_evol_text_vec_mapwork_table.q        |    2 +-
 .../schema_evol_text_vecrow_mapwork_part.q      |    2 +-
 ..._evol_text_vecrow_mapwork_part_all_complex.q |    2 +-
 ...vol_text_vecrow_mapwork_part_all_primitive.q |    2 +-
 .../schema_evol_text_vecrow_mapwork_table.q     |    2 +-
 .../clientnegative/orc_replace_columns2.q.out   |   13 +-
 .../orc_replace_columns2_acid.q.out             |   13 +-
 .../clientnegative/orc_replace_columns3.q.out   |   11 +-
 .../orc_replace_columns3_acid.q.out             |   11 +-
 .../clientnegative/orc_type_promotion1.q.out    |   13 +-
 .../orc_type_promotion1_acid.q.out              |   13 +-
 .../clientnegative/orc_type_promotion2.q.out    |   13 +-
 .../orc_type_promotion2_acid.q.out              |   13 +-
 .../clientnegative/orc_type_promotion3.q.out    |   11 +-
 .../clientnegative/orc_type_promotion3_acid.q   |   18 +
 .../orc_type_promotion3_acid.q.out              |   11 +-
 .../schema_evol_orc_acid_mapwork_part.q.out     | 4319 ++++++++++++---
 .../schema_evol_orc_acid_mapwork_table.q.out    | 3334 ++++++++++--
 .../schema_evol_orc_acidvec_mapwork_part.q.out  | 4319 ++++++++++++---
 .../schema_evol_orc_acidvec_mapwork_table.q.out | 3334 ++++++++++--
 .../schema_evol_orc_nonvec_fetchwork_part.q.out | 4905 +++++++++++++++--
 ...schema_evol_orc_nonvec_fetchwork_table.q.out | 4367 +++++++++++++++-
 .../schema_evol_orc_nonvec_mapwork_part.q.out   | 4909 +++++++++++++++--
 ...ol_orc_nonvec_mapwork_part_all_complex.q.out |  726 +++
 ..._orc_nonvec_mapwork_part_all_primitive.q.out | 2872 ++++++++++
 .../schema_evol_orc_nonvec_mapwork_table.q.out  | 4367 +++++++++++++++-
 .../schema_evol_orc_vec_mapwork_part.q.out      | 4929 ++++++++++++++++--
 ..._evol_orc_vec_mapwork_part_all_complex.q.out |  726 +++
 ...vol_orc_vec_mapwork_part_all_primitive.q.out | 2887 ++++++++++
 .../schema_evol_orc_vec_mapwork_table.q.out     | 4391 +++++++++++++++-
 .../tez/schema_evol_orc_acid_mapwork_part.q.out | 4319 ++++++++++++---
 .../schema_evol_orc_acid_mapwork_table.q.out    | 3334 ++++++++++--
 .../schema_evol_orc_acidvec_mapwork_part.q.out  | 4319 ++++++++++++---
 .../schema_evol_orc_acidvec_mapwork_table.q.out | 3334 ++++++++++--
 .../schema_evol_orc_nonvec_fetchwork_part.q.out | 4449 ++++++++++++++--
 ...schema_evol_orc_nonvec_fetchwork_table.q.out | 3911 +++++++++++++-
 .../schema_evol_orc_nonvec_mapwork_part.q.out   | 4453 ++++++++++++++--
 ...ol_orc_nonvec_mapwork_part_all_complex.q.out |  669 +++
 ..._orc_nonvec_mapwork_part_all_primitive.q.out | 2587 +++++++++
 .../schema_evol_orc_nonvec_mapwork_table.q.out  | 3911 +++++++++++++-
 .../tez/schema_evol_orc_vec_mapwork_part.q.out  | 4449 ++++++++++++++--
 ..._evol_orc_vec_mapwork_part_all_complex.q.out |  669 +++
 ...vol_orc_vec_mapwork_part_all_primitive.q.out | 2587 +++++++++
 .../tez/schema_evol_orc_vec_mapwork_table.q.out | 3911 +++++++++++++-
 service-rpc/if/TCLIService.thrift               |    6 +
 .../gen/thrift/gen-cpp/TCLIService_types.cpp    |   30 +-
 .../src/gen/thrift/gen-cpp/TCLIService_types.h  |   15 +-
 .../rpc/thrift/TExecuteStatementReq.java        |  109 +-
 .../service/rpc/thrift/TOperationState.java     |    5 +-
 service-rpc/src/gen/thrift/gen-php/Types.php    |   25 +
 .../src/gen/thrift/gen-py/TCLIService/ttypes.py |   18 +-
 .../gen/thrift/gen-rb/t_c_l_i_service_types.rb  |    9 +-
 .../org/apache/hive/service/cli/CLIService.java |   46 +-
 .../service/cli/EmbeddedCLIServiceClient.java   |   19 +-
 .../apache/hive/service/cli/ICLIService.java    |   16 +-
 .../apache/hive/service/cli/OperationState.java |    7 +-
 .../operation/ExecuteStatementOperation.java    |    9 +-
 .../cli/operation/HiveCommandOperation.java     |    5 +
 .../cli/operation/MetadataOperation.java        |    7 +-
 .../hive/service/cli/operation/Operation.java   |   17 +-
 .../service/cli/operation/OperationManager.java |   27 +-
 .../service/cli/operation/SQLOperation.java     |  106 +-
 .../hive/service/cli/session/HiveSession.java   |   28 +-
 .../service/cli/session/HiveSessionImpl.java    |   38 +-
 .../thrift/RetryingThriftCLIServiceClient.java  |   22 +-
 .../service/cli/thrift/ThriftCLIService.java    |   16 +-
 .../cli/thrift/ThriftCLIServiceClient.java      |   32 +-
 .../cli/thrift/ThriftCLIServiceTest.java        |    6 +-
 .../thrift/ThriftCliServiceTestWithCookie.java  |    3 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |  148 -
 .../org/apache/hadoop/hive/io/HdfsUtils.java    |  156 +-
 .../apache/hadoop/hive/shims/HadoopShims.java   |   41 -
 .../hadoop/hive/shims/HadoopShimsSecure.java    |   10 -
 126 files changed, 101719 insertions(+), 10497 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bc75d72b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/bc75d72b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------


[31/39] hive git commit: Merge branch 'master' into llap

Posted by jd...@apache.org.
Merge branch 'master' into llap


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e073cce2
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e073cce2
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e073cce2

Branch: refs/heads/master
Commit: e073cce2354801e12c4767f47a76fbf25168d026
Parents: 342668f e9a7218
Author: Jason Dere <jd...@hortonworks.com>
Authored: Mon May 2 18:29:03 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Mon May 2 18:29:03 2016 -0700

----------------------------------------------------------------------
 .../apache/hive/beeline/DatabaseConnection.java |   12 +-
 .../org/apache/hadoop/hive/cli/CliDriver.java   |    5 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   19 +
 data/files/keystore.jks                         |  Bin 2248 -> 2206 bytes
 data/files/keystore_exampledotcom.jks           |  Bin 0 -> 2222 bytes
 data/files/struct1_a.txt                        |    4 +
 data/files/struct1_b.txt                        |    1 +
 data/files/struct1_c.txt                        |    1 +
 data/files/struct2_a.txt                        |    2 +
 data/files/struct2_b.txt                        |    2 +
 data/files/struct2_c.txt                        |    1 +
 data/files/struct2_d.txt                        |    1 +
 data/files/struct3_a.txt                        |    2 +
 data/files/struct3_b.txt                        |    1 +
 data/files/struct3_c.txt                        |    1 +
 data/files/struct4_a.txt                        |    2 +
 data/files/struct4_b.txt                        |    1 +
 data/files/struct4_c.txt                        |    1 +
 data/files/truststore.jks                       |  Bin 958 -> 1816 bytes
 .../hcatalog/templeton/ExecServiceImpl.java     |    7 +-
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |   35 +-
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |   25 +-
 .../test/java/org/apache/hive/jdbc/TestSSL.java |   71 +-
 itests/pom.xml                                  |    3 -
 .../test/resources/testconfiguration.properties |   34 +-
 .../org/apache/hive/jdbc/HiveConnection.java    |   99 +-
 .../java/org/apache/hive/jdbc/JdbcColumn.java   |   13 +-
 .../hadoop/hive/llap/cli/LlapServiceDriver.java |   21 +-
 .../llap/cli/LlapStatusOptionsProcessor.java    |   44 +-
 .../hive/llap/cli/LlapStatusServiceDriver.java  |   74 +-
 .../daemon/impl/LlapProtocolServerImpl.java     |   57 +-
 .../hive/llap/io/api/impl/LlapInputFormat.java  |    2 +-
 .../main/resources/llap-cli-log4j2.properties   |    2 +-
 llap-server/src/main/resources/package.py       |    7 +-
 packaging/src/main/assembly/bin.xml             |    5 +
 .../hive/ql/exec/AbstractMapOperator.java       |  178 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    7 +
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |    8 +
 .../apache/hadoop/hive/ql/exec/MapOperator.java |   86 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |   33 +-
 .../hadoop/hive/ql/exec/mr/ExecMapper.java      |    3 +-
 .../ql/exec/spark/SparkMapRecordHandler.java    |    3 +-
 .../hadoop/hive/ql/exec/tez/DagUtils.java       |    3 +-
 .../hive/ql/exec/tez/MapRecordProcessor.java    |   15 +-
 .../hive/ql/exec/tez/MapRecordSource.java       |    6 +-
 .../hive/ql/exec/tez/ReduceRecordSource.java    |    4 +-
 .../vector/VectorAppMasterEventOperator.java    |   16 +-
 .../hive/ql/exec/vector/VectorAssignRow.java    | 1111 ++--
 .../ql/exec/vector/VectorAssignRowDynBatch.java |   41 -
 .../exec/vector/VectorAssignRowSameBatch.java   |   36 -
 .../ql/exec/vector/VectorDeserializeRow.java    | 1114 ++--
 .../hive/ql/exec/vector/VectorExtractRow.java   |  971 +---
 .../exec/vector/VectorExtractRowDynBatch.java   |   40 -
 .../exec/vector/VectorExtractRowSameBatch.java  |   36 -
 .../ql/exec/vector/VectorFileSinkOperator.java  |   16 +-
 .../ql/exec/vector/VectorGroupByOperator.java   |   13 +-
 .../exec/vector/VectorMapJoinBaseOperator.java  |   11 +-
 .../ql/exec/vector/VectorMapJoinOperator.java   |    4 +-
 .../VectorMapJoinOuterFilteredOperator.java     |   17 +-
 .../hive/ql/exec/vector/VectorMapOperator.java  |  848 ++-
 .../exec/vector/VectorReduceSinkOperator.java   |   16 +-
 .../exec/vector/VectorSMBMapJoinOperator.java   |   11 +-
 .../VectorSparkHashTableSinkOperator.java       |   16 +-
 ...VectorSparkPartitionPruningSinkOperator.java |   13 +-
 .../ql/exec/vector/VectorizationContext.java    |   12 +-
 .../ql/exec/vector/VectorizedBatchUtil.java     |   49 +
 .../VectorMapJoinGenerateResultOperator.java    |    8 +-
 .../fast/VectorMapJoinFastLongHashUtil.java     |   10 +-
 .../fast/VectorMapJoinFastStringCommon.java     |   10 +-
 .../hadoop/hive/ql/io/HiveInputFormat.java      |    6 +-
 .../hadoop/hive/ql/io/NullRowsInputFormat.java  |    2 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |    4 +-
 .../ql/io/parquet/MapredParquetInputFormat.java |    2 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |  334 +-
 .../hadoop/hive/ql/parse/FromClauseParser.g     |    4 +-
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |    6 +-
 .../hadoop/hive/ql/parse/SelectClauseParser.g   |    4 +-
 .../apache/hadoop/hive/ql/plan/BaseWork.java    |   23 +
 .../org/apache/hadoop/hive/ql/plan/MapWork.java |   11 +
 .../org/apache/hadoop/hive/ql/plan/TezWork.java |    9 +-
 .../hive/ql/plan/VectorPartitionConversion.java |  172 +-
 .../hive/ql/plan/VectorPartitionDesc.java       |  164 +-
 .../hadoop/hive/ql/udf/generic/BaseMaskUDF.java |  473 ++
 .../hive/ql/udf/generic/GenericUDFMask.java     |  334 ++
 .../ql/udf/generic/GenericUDFMaskFirstN.java    |  229 +
 .../hive/ql/udf/generic/GenericUDFMaskHash.java |   77 +
 .../ql/udf/generic/GenericUDFMaskLastN.java     |  193 +
 .../udf/generic/GenericUDFMaskShowFirstN.java   |  248 +
 .../ql/udf/generic/GenericUDFMaskShowLastN.java |  198 +
 .../ql/exec/vector/TestVectorRowObject.java     |   14 +-
 .../hive/ql/exec/vector/TestVectorSerDeRow.java |  169 +-
 .../hive/ql/io/orc/TestInputOutputFormat.java   |   10 +-
 .../hive/ql/plan/TestTezWorkConcurrency.java    |   65 +
 .../avro_schema_evolution_native.q              |   18 +
 .../queries/clientpositive/bucket_groupby.q     |   33 +-
 .../queries/clientpositive/groupby_sort_10.q    |    2 +
 .../schema_evol_orc_acidvec_mapwork_part.q      |    3 +
 .../schema_evol_orc_acidvec_mapwork_table.q     |    3 +
 .../schema_evol_orc_nonvec_mapwork_table.q      |    2 -
 .../schema_evol_orc_vec_mapwork_part.q          |    3 +
 .../schema_evol_orc_vec_mapwork_table.q         |    7 +-
 .../schema_evol_text_fetchwork_table.q          |   56 -
 .../schema_evol_text_mapwork_table.q            |   56 -
 .../schema_evol_text_nonvec_fetchwork_part.q    |   98 -
 .../schema_evol_text_nonvec_fetchwork_table.q   |   67 -
 .../schema_evol_text_nonvec_mapwork_part.q      |  828 ++-
 ..._evol_text_nonvec_mapwork_part_all_complex.q |  159 +
 ...vol_text_nonvec_mapwork_part_all_primitive.q |  509 ++
 .../schema_evol_text_nonvec_mapwork_table.q     |  822 ++-
 .../schema_evol_text_vec_mapwork_part.q         |  827 +++
 ...ema_evol_text_vec_mapwork_part_all_complex.q |  164 +
 ...a_evol_text_vec_mapwork_part_all_primitive.q |  514 ++
 .../schema_evol_text_vec_mapwork_table.q        |  826 +++
 .../schema_evol_text_vecrow_mapwork_part.q      |  827 +++
 ..._evol_text_vecrow_mapwork_part_all_complex.q |  165 +
 ...vol_text_vecrow_mapwork_part_all_primitive.q |  514 ++
 .../schema_evol_text_vecrow_mapwork_table.q     |  826 +++
 .../clientpositive/tez_schema_evolution.q       |    1 +
 ql/src/test/queries/clientpositive/udf_mask.q   |   13 +
 .../queries/clientpositive/udf_mask_first_n.q   |   13 +
 .../test/queries/clientpositive/udf_mask_hash.q |   13 +
 .../queries/clientpositive/udf_mask_last_n.q    |   13 +
 .../clientpositive/udf_mask_show_first_n.q      |   13 +
 .../clientpositive/udf_mask_show_last_n.q       |   13 +
 .../beelinepositive/show_functions.q.out        |    6 +
 .../results/clientnegative/subq_insert.q.out    |    2 +-
 .../avro_schema_evolution_native.q.out          |  206 +
 .../results/clientpositive/bucket_groupby.q.out |  308 +-
 .../results/clientpositive/groupby1_limit.q.out |    2 +-
 .../clientpositive/groupby_sort_10.q.out        |    8 +-
 .../results/clientpositive/regexp_extract.q.out |  137 -
 .../schema_evol_text_fetchwork_table.q.out      |  298 --
 .../schema_evol_text_mapwork_table.q.out        |  298 --
 ...schema_evol_text_nonvec_fetchwork_part.q.out |  642 ---
 ...chema_evol_text_nonvec_fetchwork_table.q.out |  297 --
 .../schema_evol_text_nonvec_mapwork_part.q.out  | 4909 ++++++++++++++++--
 ...l_text_nonvec_mapwork_part_all_complex.q.out |  726 +++
 ...text_nonvec_mapwork_part_all_primitive.q.out | 3038 +++++++++++
 .../schema_evol_text_nonvec_mapwork_table.q.out | 4376 +++++++++++++++-
 .../schema_evol_text_vec_mapwork_part.q.out     | 4479 ++++++++++++++++
 ...evol_text_vec_mapwork_part_all_complex.q.out |  730 +++
 ...ol_text_vec_mapwork_part_all_primitive.q.out | 3058 +++++++++++
 .../schema_evol_text_vec_mapwork_table.q.out    | 4221 +++++++++++++++
 .../schema_evol_text_vecrow_mapwork_part.q.out  | 4479 ++++++++++++++++
 ...l_text_vecrow_mapwork_part_all_complex.q.out |  732 +++
 ...text_vecrow_mapwork_part_all_primitive.q.out | 3058 +++++++++++
 .../schema_evol_text_vecrow_mapwork_table.q.out | 4221 +++++++++++++++
 .../results/clientpositive/show_functions.q.out |   12 +
 .../tez/schema_evol_text_fetchwork_table.q.out  |  298 --
 .../tez/schema_evol_text_mapwork_table.q.out    |  298 --
 ...schema_evol_text_nonvec_fetchwork_part.q.out |  642 ---
 ...chema_evol_text_nonvec_fetchwork_table.q.out |  297 --
 .../schema_evol_text_nonvec_mapwork_part.q.out  | 4453 ++++++++++++++--
 ...l_text_nonvec_mapwork_part_all_complex.q.out |  669 +++
 ...text_nonvec_mapwork_part_all_primitive.q.out | 2734 ++++++++++
 .../schema_evol_text_nonvec_mapwork_table.q.out | 3920 +++++++++++++-
 .../tez/schema_evol_text_vec_mapwork_part.q.out | 3999 ++++++++++++++
 ...evol_text_vec_mapwork_part_all_complex.q.out |  673 +++
 ...ol_text_vec_mapwork_part_all_primitive.q.out | 2738 ++++++++++
 .../schema_evol_text_vec_mapwork_table.q.out    | 3741 +++++++++++++
 .../schema_evol_text_vecrow_mapwork_part.q.out  | 3999 ++++++++++++++
 ...l_text_vecrow_mapwork_part_all_complex.q.out |  675 +++
 ...text_vecrow_mapwork_part_all_primitive.q.out | 2738 ++++++++++
 .../schema_evol_text_vecrow_mapwork_table.q.out | 3741 +++++++++++++
 .../vector_orc_string_reader_empty_dict.q.out   |   62 +
 .../tez/vector_partition_diff_num_cols.q.out    |    1 +
 .../tez/vector_tablesample_rows.q.out           |  307 ++
 .../test/results/clientpositive/udf_mask.q.out  |   71 +
 .../clientpositive/udf_mask_first_n.q.out       |   68 +
 .../results/clientpositive/udf_mask_hash.q.out  |   59 +
 .../clientpositive/udf_mask_last_n.q.out        |   68 +
 .../clientpositive/udf_mask_show_first_n.q.out  |   68 +
 .../clientpositive/udf_mask_show_last_n.q.out   |   68 +
 .../vector_partition_diff_num_cols.q.out        |    1 +
 .../vector_tablesample_rows.q.out               |    2 -
 .../compiler/errors/missing_overwrite.q.out     |    2 +-
 .../fast/BinarySortableDeserializeRead.java     |  806 +--
 .../hive/serde2/fast/DeserializeRead.java       |  379 +-
 .../lazy/fast/LazySimpleDeserializeRead.java    |  704 +--
 .../fast/LazyBinaryDeserializeRead.java         |  944 +---
 .../apache/hadoop/hive/serde2/VerifyFast.java   |   75 +-
 .../hive/service/auth/HiveAuthFactory.java      |   21 +-
 182 files changed, 84895 insertions(+), 10148 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e073cce2/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --cc common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 7db492f,caadf2a..5f78ae6
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@@ -2658,13 -2670,17 +2670,20 @@@ public class HiveConf extends Configura
          "ZooKeeper for ZooKeeper SecretManager."),
      LLAP_ZKSM_ZK_CONNECTION_STRING("hive.llap.zk.sm.connectionString", "",
          "ZooKeeper connection string for ZooKeeper SecretManager."),
 +    LLAP_ZK_REGISTRY_USER("hive.llap.zk.registry.user", "",
 +        "In the LLAP ZooKeeper-based registry, specifies the username in the Zookeeper path.\n" +
 +        "This should be the hive user or whichever user is running the LLAP daemon."),
      // Note: do not rename to ..service.acl; Hadoop generates .hosts setting name from this,
      // resulting in a collision with existing hive.llap.daemon.service.hosts and bizarre errors.
+     // These are read by Hadoop IPC, so you should check the usage and naming conventions (e.g.
+     // ".blocked" is a string hardcoded by Hadoop, and defaults are enforced elsewhere in Hive)
+     // before making changes or copy-pasting these.
      LLAP_SECURITY_ACL("hive.llap.daemon.acl", "*", "The ACL for LLAP daemon."),
+     LLAP_SECURITY_ACL_DENY("hive.llap.daemon.acl.blocked", "", "The deny ACL for LLAP daemon."),
      LLAP_MANAGEMENT_ACL("hive.llap.management.acl", "*", "The ACL for LLAP daemon management."),
+     LLAP_MANAGEMENT_ACL_DENY("hive.llap.management.acl.blocked", "",
+         "The deny ACL for LLAP daemon management."),
+ 
      // Hadoop DelegationTokenManager default is 1 week.
      LLAP_DELEGATION_TOKEN_LIFETIME("hive.llap.daemon.delegation.token.lifetime", "14d",
           new TimeValidator(TimeUnit.SECONDS),

http://git-wip-us.apache.org/repos/asf/hive/blob/e073cce2/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/e073cce2/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
index 7a3d6a6,9a9f43a..dc63d7b
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
@@@ -33,9 -33,9 +33,10 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  import org.apache.hadoop.conf.Configuration;
  import org.apache.hadoop.hive.conf.HiveConf;
+ import org.apache.hadoop.hive.ql.exec.AbstractMapOperator;
  import org.apache.hadoop.hive.llap.io.api.LlapProxy;
  import org.apache.hadoop.hive.ql.CompilationOpContext;
 +import org.apache.hadoop.hive.llap.LlapOutputFormat;
  import org.apache.hadoop.hive.ql.exec.DummyStoreOperator;
  import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator;
  import org.apache.hadoop.hive.ql.exec.MapOperator;


[36/39] hive git commit: HIVE-13695: LlapOutputFormatService port should be able to be set via conf

Posted by jd...@apache.org.
HIVE-13695: LlapOutputFormatService port should be able to be set via conf


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/03ee0481
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/03ee0481
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/03ee0481

Branch: refs/heads/master
Commit: 03ee0481a518585a4a92875d88c560ff525d75d4
Parents: 2a03f1f
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu May 5 12:56:20 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu May 5 12:56:20 2016 -0700

----------------------------------------------------------------------
 .../hive/llap/daemon/impl/LlapDaemon.java       |  6 +++
 .../hive/llap/daemon/MiniLlapCluster.java       |  3 ++
 .../hive/llap/LlapOutputFormatService.java      | 44 +++++++++++++-------
 3 files changed, 38 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/03ee0481/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 223c390..b3c1abf 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
@@ -132,6 +132,10 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
         "Work dirs must be specified");
     Preconditions.checkArgument(shufflePort == 0 || (shufflePort > 1024 && shufflePort < 65536),
         "Shuffle Port must be betwee 1024 and 65535, or 0 for automatic selection");
+    int outputFormatServicePort = HiveConf.getIntVar(daemonConf, HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
+    Preconditions.checkArgument(outputFormatServicePort == 0
+        || (outputFormatServicePort > 1024 && outputFormatServicePort < 65536),
+        "OutputFormatService Port must be between 1024 and 65535, or 0 for automatic selection");
     String hosts = HiveConf.getTrimmedVar(daemonConf, ConfVars.LLAP_DAEMON_SERVICE_HOSTS);
     if (hosts.startsWith("@")) {
       String zkHosts = HiveConf.getTrimmedVar(daemonConf, ConfVars.HIVE_ZOOKEEPER_QUORUM);
@@ -165,6 +169,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
         ", rpcListenerPort=" + srvPort +
         ", mngListenerPort=" + mngPort +
         ", webPort=" + webPort +
+        ", outputFormatSvcPort=" + outputFormatServicePort +
         ", workDirs=" + Arrays.toString(localDirs) +
         ", shufflePort=" + shufflePort +
         ", executorMemory=" + executorMemoryBytes +
@@ -335,6 +340,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     this.shufflePort.set(ShuffleHandler.get().getPort());
     getConfig()
         .setInt(ConfVars.LLAP_DAEMON_YARN_SHUFFLE_PORT.varname, ShuffleHandler.get().getPort());
+    LlapOutputFormatService.initializeAndStart(getConfig());
     super.serviceStart();
 
     // Setup the actual ports in the configuration.

http://git-wip-us.apache.org/repos/asf/hive/blob/03ee0481/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 dde5be0..e394191 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
@@ -166,6 +166,7 @@ public class MiniLlapCluster extends AbstractService {
     int mngPort = 0;
     int shufflePort = 0;
     int webPort = 0;
+    int outputFormatServicePort = 0;
     boolean usePortsFromConf = conf.getBoolean("minillap.usePortsFromConf", false);
     LOG.info("MiniLlap configured to use ports from conf: {}", usePortsFromConf);
     if (usePortsFromConf) {
@@ -173,7 +174,9 @@ public class MiniLlapCluster extends AbstractService {
       mngPort = HiveConf.getIntVar(conf, HiveConf.ConfVars.LLAP_MANAGEMENT_RPC_PORT);
       shufflePort = conf.getInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, ShuffleHandler.DEFAULT_SHUFFLE_PORT);
       webPort = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_WEB_PORT);
+      outputFormatServicePort = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
     }
+    HiveConf.setIntVar(conf, ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT, outputFormatServicePort);
 
     if (ownZkCluster) {
       miniZooKeeperCluster = new MiniZooKeeperCluster();

http://git-wip-us.apache.org/repos/asf/hive/blob/03ee0481/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
index 6adbf7c..f852041 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
@@ -25,6 +25,7 @@ import java.net.InetSocketAddress;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.BytesWritable;
@@ -39,9 +40,13 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
+
+import com.google.common.base.Preconditions;
+
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.channel.ChannelFuture;
@@ -67,9 +72,12 @@ public class LlapOutputFormatService {
 
   private static final Logger LOG = LoggerFactory.getLogger(LlapOutputFormat.class);
 
-  private static LlapOutputFormatService service;
+  private static final AtomicBoolean started = new AtomicBoolean(false);
+  private static final AtomicBoolean initing = new AtomicBoolean(false);
+  private static LlapOutputFormatService INSTANCE;
+
   private final Map<String, RecordWriter> writers;
-  private final HiveConf conf;
+  private final Configuration conf;
   private static final int WAIT_TIME = 5;
   private static final int MAX_QUERY_ID_LENGTH = 256;
 
@@ -78,23 +86,29 @@ public class LlapOutputFormatService {
   private ChannelFuture listeningChannelFuture;
   private int port;
 
-  private LlapOutputFormatService() throws IOException {
+  private LlapOutputFormatService(Configuration conf) throws IOException {
     writers = new HashMap<String, RecordWriter>();
-    conf = new HiveConf();
+    this.conf = conf;
   }
 
-  public static LlapOutputFormatService get() throws IOException {
-    if (service == null) {
-      service = new LlapOutputFormatService();
-      service.start();
+  public static void initializeAndStart(Configuration conf) throws Exception {
+    if (!initing.getAndSet(true)) {
+      INSTANCE = new LlapOutputFormatService(conf);
+      INSTANCE.start();
+      started.set(true);
     }
-    return service;
+  }
+
+  public static LlapOutputFormatService get() throws IOException {
+    Preconditions.checkState(started.get(),
+        "LlapOutputFormatService must be started before invoking get");
+    return INSTANCE;
   }
 
   public void start() throws IOException {
     LOG.info("Starting LlapOutputFormatService");
 
-    int portFromConf = conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
+    int portFromConf = HiveConf.getIntVar(conf, HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
     eventLoopGroup = new NioEventLoopGroup(1);
     serverBootstrap = new ServerBootstrap();
     serverBootstrap.group(eventLoopGroup);
@@ -125,10 +139,10 @@ public class LlapOutputFormatService {
 
   public <K,V> RecordWriter<K, V> getWriter(String id) throws IOException, InterruptedException {
     RecordWriter writer = null;
-    synchronized(service) {
+    synchronized(INSTANCE) {
       while ((writer = writers.get(id)) == null) {
         LOG.info("Waiting for writer for: "+id);
-        service.wait();
+        INSTANCE.wait();
       }
     }
     LOG.info("Returning writer for: "+id);
@@ -147,7 +161,7 @@ public class LlapOutputFormatService {
     }
 
     private void registerReader(ChannelHandlerContext ctx, String id) {
-      synchronized(service) {
+      synchronized(INSTANCE) {
         LOG.debug("registering socket for: "+id);
         int bufSize = 128 * 1024; // configable?
         OutputStream stream = new ChannelOutputStream(ctx, id, bufSize);
@@ -157,7 +171,7 @@ public class LlapOutputFormatService {
         // Add listener to handle any cleanup for when the connection is closed
         ctx.channel().closeFuture().addListener(new LlapOutputFormatChannelCloseListener(id));
 
-        service.notifyAll();
+        INSTANCE.notifyAll();
       }
     }
   }
@@ -173,7 +187,7 @@ public class LlapOutputFormatService {
     public void operationComplete(ChannelFuture future) throws Exception {
       RecordWriter writer = null;
 
-      synchronized (service) {
+      synchronized (INSTANCE) {
         writer = writers.remove(id);
       }
 


[28/39] hive git commit: HIVE-13672: Use loginUser from UGI to get llap user when generating LLAP splits.

Posted by jd...@apache.org.
HIVE-13672: Use loginUser from UGI to get llap user when generating LLAP splits.


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/6d1b6bb4
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/6d1b6bb4
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/6d1b6bb4

Branch: refs/heads/master
Commit: 6d1b6bb40ab4b6ebedc721ed76d9a17a200ccca9
Parents: 9f999f2
Author: Jason Dere <jd...@hortonworks.com>
Authored: Mon May 2 15:07:24 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Mon May 2 15:07:24 2016 -0700

----------------------------------------------------------------------
 ql/pom.xml                                                     | 6 ------
 .../hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java       | 4 ++--
 2 files changed, 2 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6d1b6bb4/ql/pom.xml
----------------------------------------------------------------------
diff --git a/ql/pom.xml b/ql/pom.xml
index 1f63b9f..aaa3271 100644
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@ -291,12 +291,6 @@
       <version>${hadoop.version}</version>
       <optional>true</optional>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-registry</artifactId>
-      <version>${hadoop.version}</version>
-      <optional>true</optional>
-    </dependency>
 
     <dependency>
       <groupId>org.apache.ivy</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/6d1b6bb4/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
index 2d36e5c..50cdadb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
@@ -84,7 +84,7 @@ import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SplitLocationInfo;
-import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
@@ -323,7 +323,7 @@ public class GenericUDTFGetSplits extends GenericUDTF {
       ApplicationId fakeApplicationId
         = ApplicationId.newInstance(Math.abs(new Random().nextInt()), 0);
 
-      String llapUser = RegistryUtils.currentUser();
+      String llapUser = UserGroupInformation.getLoginUser().getShortUserName();
       LOG.info("Number of splits: " + (eventList.size() - 1));
       for (int i = 0; i < eventList.size() - 1; i++) {
 


[19/39] hive git commit: HIVE-13529: Move around some of the classes created during llap branch work

Posted by jd...@apache.org.
HIVE-13529: Move around some of the classes created during llap branch work


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7b9096a9
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7b9096a9
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7b9096a9

Branch: refs/heads/master
Commit: 7b9096a922f9706909ba0e52d8188d182a79612f
Parents: fc7343d
Author: Jason Dere <jd...@hortonworks.com>
Authored: Fri Apr 15 16:45:32 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Fri Apr 15 16:45:32 2016 -0700

----------------------------------------------------------------------
 itests/hive-unit/pom.xml                        |   5 +
 .../hadoop/hive/jdbc/TestLlapInputSplit.java    | 100 -----
 .../hive/llap/ext/TestLlapInputSplit.java       | 100 +++++
 .../apache/hive/jdbc/TestJdbcWithMiniLlap.java  |   4 +-
 .../apache/hive/jdbc/LlapBaseInputFormat.java   | 135 ------
 .../src/java/org/apache/hive/jdbc/LlapDump.java | 164 --------
 .../org/apache/hive/jdbc/LlapInputSplit.java    |  73 ----
 .../apache/hive/jdbc/LlapRowInputFormat.java    |  34 --
 llap-client/pom.xml                             |  32 ++
 .../hadoop/hive/llap/LlapBaseRecordReader.java  | 205 +++++++++
 .../hadoop/hive/llap/LlapInputFormat.java       | 392 ++++++++++++++++++
 .../apache/hadoop/hive/llap/LlapInputSplit.java | 131 ++++++
 .../hadoop/hive/llap/LlapRowRecordReader.java   | 155 +++++++
 .../apache/hadoop/hive/llap/SubmitWorkInfo.java | 103 +++++
 .../ext/LlapTaskUmbilicalExternalClient.java    | 415 +++++++++++++++++++
 .../helpers/LlapTaskUmbilicalServer.java        |  57 +++
 .../hadoop/hive/llap/LlapRowRecordReader.java   | 155 -------
 llap-ext-client/pom.xml                         | 140 +++++++
 .../hadoop/hive/llap/LlapBaseInputFormat.java   | 136 ++++++
 .../org/apache/hadoop/hive/llap/LlapDump.java   | 165 ++++++++
 .../hadoop/hive/llap/LlapRowInputFormat.java    |  36 ++
 .../apache/hive/llap/ext/LlapInputSplit.java    |  73 ++++
 .../hadoop/hive/llap/LlapInputFormat.java       | 392 ------------------
 .../ext/LlapTaskUmbilicalExternalClient.java    | 415 -------------------
 .../helpers/LlapTaskUmbilicalServer.java        |  57 ---
 pom.xml                                         |   1 +
 .../hadoop/hive/llap/LlapBaseRecordReader.java  | 205 ---------
 .../apache/hadoop/hive/llap/LlapInputSplit.java | 131 ------
 .../apache/hadoop/hive/llap/SubmitWorkInfo.java | 103 -----
 29 files changed, 2148 insertions(+), 1966 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/itests/hive-unit/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml
index ae231de..b248673 100644
--- a/itests/hive-unit/pom.xml
+++ b/itests/hive-unit/pom.xml
@@ -61,6 +61,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
+      <artifactId>hive-llap-ext-client</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
       <artifactId>hive-llap-server</artifactId>
       <version>${project.version}</version>
       <type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
deleted file mode 100644
index 366e326..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
+++ /dev/null
@@ -1,100 +0,0 @@
-package org.apache.hive.jdbc;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.util.ArrayList;
-import java.util.HashMap;
-
-import org.apache.hadoop.io.Text;
-
-import org.apache.hadoop.hive.llap.Schema;
-import org.apache.hadoop.hive.llap.FieldDesc;
-import org.apache.hadoop.hive.llap.TypeDesc;
-
-import org.apache.hadoop.mapred.SplitLocationInfo;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import static org.junit.Assert.*;
-
-public class TestLlapInputSplit {
-
-  @Test
-  public void testWritable() throws Exception {
-    int splitNum = 88;
-    byte[] planBytes = "0123456789987654321".getBytes();
-    byte[] fragmentBytes = "abcdefghijklmnopqrstuvwxyz".getBytes();
-    SplitLocationInfo[] locations = {
-        new SplitLocationInfo("location1", false),
-        new SplitLocationInfo("location2", false),
-    };
-
-    ArrayList<FieldDesc> colDescs = new ArrayList<FieldDesc>();
-    colDescs.add(new FieldDesc("col1", new TypeDesc(TypeDesc.Type.STRING)));
-    colDescs.add(new FieldDesc("col2", new TypeDesc(TypeDesc.Type.INT)));
-    Schema schema = new Schema(colDescs);
-
-    org.apache.hadoop.hive.llap.LlapInputSplit split1 = new org.apache.hadoop.hive.llap.LlapInputSplit(
-        splitNum,
-        planBytes,
-        fragmentBytes,
-        locations,
-        schema,
-        "hive");
-    ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream();
-    DataOutputStream dataOut = new DataOutputStream(byteOutStream);
-    split1.write(dataOut);
-    ByteArrayInputStream byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray());
-    DataInputStream dataIn = new DataInputStream(byteInStream);
-    org.apache.hadoop.hive.llap.LlapInputSplit split2 = new org.apache.hadoop.hive.llap.LlapInputSplit();
-    split2.readFields(dataIn);
-
-    // Did we read all the data?
-    assertEquals(0, byteInStream.available());
-
-    checkLlapSplits(split1, split2);
-
-    // Try JDBC LlapInputSplits
-    org.apache.hive.jdbc.LlapInputSplit<Text> jdbcSplit1 =
-        new org.apache.hive.jdbc.LlapInputSplit<Text>(split1, "org.apache.hadoop.hive.llap.LlapInputFormat");
-    byteOutStream.reset();
-    jdbcSplit1.write(dataOut);
-    byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray());
-    dataIn = new DataInputStream(byteInStream);
-    org.apache.hive.jdbc.LlapInputSplit<Text> jdbcSplit2 = new org.apache.hive.jdbc.LlapInputSplit<Text>();
-    jdbcSplit2.readFields(dataIn);
-
-    assertEquals(0, byteInStream.available());
-
-    checkLlapSplits(
-        (org.apache.hadoop.hive.llap.LlapInputSplit) jdbcSplit1.getSplit(),
-        (org.apache.hadoop.hive.llap.LlapInputSplit) jdbcSplit2.getSplit());
-    assertEquals(jdbcSplit1.getInputFormat().getClass(), jdbcSplit2.getInputFormat().getClass());
-  }
-
-  static void checkLlapSplits(
-      org.apache.hadoop.hive.llap.LlapInputSplit split1,
-      org.apache.hadoop.hive.llap.LlapInputSplit split2) throws Exception {
-
-    assertEquals(split1.getSplitNum(), split2.getSplitNum());
-    assertArrayEquals(split1.getPlanBytes(), split2.getPlanBytes());
-    assertArrayEquals(split1.getFragmentBytes(), split2.getFragmentBytes());
-    SplitLocationInfo[] locationInfo1 = split1.getLocationInfo();
-    SplitLocationInfo[] locationInfo2 = split2.getLocationInfo();
-    for (int idx = 0; idx < locationInfo1.length; ++idx) {
-      assertEquals(locationInfo1[idx].getLocation(), locationInfo2[idx].getLocation());
-      assertEquals(locationInfo1[idx].isInMemory(), locationInfo2[idx].isInMemory());
-      assertEquals(locationInfo1[idx].isOnDisk(), locationInfo2[idx].isOnDisk());
-    }
-    assertArrayEquals(split1.getLocations(), split2.getLocations());
-    assertEquals(split1.getSchema().toString(), split2.getSchema().toString());
-    assertEquals(split1.getLlapUser(), split2.getLlapUser());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
new file mode 100644
index 0000000..04da17e
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
@@ -0,0 +1,100 @@
+package org.apache.hadoop.hive.llap.ext;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+
+import org.apache.hadoop.io.Text;
+
+import org.apache.hadoop.hive.llap.Schema;
+import org.apache.hadoop.hive.llap.FieldDesc;
+import org.apache.hadoop.hive.llap.TypeDesc;
+
+import org.apache.hadoop.mapred.SplitLocationInfo;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import static org.junit.Assert.*;
+
+public class TestLlapInputSplit {
+
+  @Test
+  public void testWritable() throws Exception {
+    int splitNum = 88;
+    byte[] planBytes = "0123456789987654321".getBytes();
+    byte[] fragmentBytes = "abcdefghijklmnopqrstuvwxyz".getBytes();
+    SplitLocationInfo[] locations = {
+        new SplitLocationInfo("location1", false),
+        new SplitLocationInfo("location2", false),
+    };
+
+    ArrayList<FieldDesc> colDescs = new ArrayList<FieldDesc>();
+    colDescs.add(new FieldDesc("col1", new TypeDesc(TypeDesc.Type.STRING)));
+    colDescs.add(new FieldDesc("col2", new TypeDesc(TypeDesc.Type.INT)));
+    Schema schema = new Schema(colDescs);
+
+    org.apache.hadoop.hive.llap.LlapInputSplit split1 = new org.apache.hadoop.hive.llap.LlapInputSplit(
+        splitNum,
+        planBytes,
+        fragmentBytes,
+        locations,
+        schema,
+        "hive");
+    ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream();
+    DataOutputStream dataOut = new DataOutputStream(byteOutStream);
+    split1.write(dataOut);
+    ByteArrayInputStream byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray());
+    DataInputStream dataIn = new DataInputStream(byteInStream);
+    org.apache.hadoop.hive.llap.LlapInputSplit split2 = new org.apache.hadoop.hive.llap.LlapInputSplit();
+    split2.readFields(dataIn);
+
+    // Did we read all the data?
+    assertEquals(0, byteInStream.available());
+
+    checkLlapSplits(split1, split2);
+
+    // Try JDBC LlapInputSplits
+    org.apache.hive.llap.ext.LlapInputSplit<Text> jdbcSplit1 =
+        new org.apache.hive.llap.ext.LlapInputSplit<Text>(split1, "org.apache.hadoop.hive.llap.LlapInputFormat");
+    byteOutStream.reset();
+    jdbcSplit1.write(dataOut);
+    byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray());
+    dataIn = new DataInputStream(byteInStream);
+    org.apache.hive.llap.ext.LlapInputSplit<Text> jdbcSplit2 = new org.apache.hive.llap.ext.LlapInputSplit<Text>();
+    jdbcSplit2.readFields(dataIn);
+
+    assertEquals(0, byteInStream.available());
+
+    checkLlapSplits(
+        (org.apache.hadoop.hive.llap.LlapInputSplit) jdbcSplit1.getSplit(),
+        (org.apache.hadoop.hive.llap.LlapInputSplit) jdbcSplit2.getSplit());
+    assertEquals(jdbcSplit1.getInputFormat().getClass(), jdbcSplit2.getInputFormat().getClass());
+  }
+
+  static void checkLlapSplits(
+      org.apache.hadoop.hive.llap.LlapInputSplit split1,
+      org.apache.hadoop.hive.llap.LlapInputSplit split2) throws Exception {
+
+    assertEquals(split1.getSplitNum(), split2.getSplitNum());
+    assertArrayEquals(split1.getPlanBytes(), split2.getPlanBytes());
+    assertArrayEquals(split1.getFragmentBytes(), split2.getFragmentBytes());
+    SplitLocationInfo[] locationInfo1 = split1.getLocationInfo();
+    SplitLocationInfo[] locationInfo2 = split2.getLocationInfo();
+    for (int idx = 0; idx < locationInfo1.length; ++idx) {
+      assertEquals(locationInfo1[idx].getLocation(), locationInfo2[idx].getLocation());
+      assertEquals(locationInfo1[idx].isInMemory(), locationInfo2[idx].isInMemory());
+      assertEquals(locationInfo1[idx].isOnDisk(), locationInfo2[idx].isOnDisk());
+    }
+    assertArrayEquals(split1.getLocations(), split2.getLocations());
+    assertEquals(split1.getSchema().toString(), split2.getSchema().toString());
+    assertEquals(split1.getLlapUser(), split2.getLlapUser());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
index deeac2e..5b4ba49 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
@@ -68,8 +68,8 @@ import org.apache.hadoop.io.Text;
 
 import org.apache.hive.jdbc.miniHS2.MiniHS2;
 import org.apache.hive.jdbc.miniHS2.MiniHS2.MiniClusterType;
-import org.apache.hive.jdbc.LlapBaseInputFormat;
-import org.apache.hive.jdbc.LlapRowInputFormat;
+import org.apache.hadoop.hive.llap.LlapBaseInputFormat;
+import org.apache.hadoop.hive.llap.LlapRowInputFormat;
 
 import org.datanucleus.ClassLoaderResolver;
 import org.datanucleus.NucleusContext;

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/jdbc/src/java/org/apache/hive/jdbc/LlapBaseInputFormat.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapBaseInputFormat.java b/jdbc/src/java/org/apache/hive/jdbc/LlapBaseInputFormat.java
deleted file mode 100644
index a0ddeaa..0000000
--- a/jdbc/src/java/org/apache/hive/jdbc/LlapBaseInputFormat.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.jdbc;
-
-import java.util.ArrayList;
-import java.util.List;
-
-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 org.apache.hadoop.fs.FileSystem;
-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.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.util.Progressable;
-
-import com.google.common.base.Preconditions;
-
-public class LlapBaseInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
-
-  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;
-    return llapSplit.getInputFormat().getRecordReader(llapSplit.getSplit(), job, reporter);
-  }
-
-  @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(3));
-        InputSplitWithLocationInfo is = (InputSplitWithLocationInfo)Class.forName(res.getString(2)).newInstance();
-        is.readFields(in);
-        ins.add(new LlapInputSplit(is, res.getString(1)));
-      }
-
-      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
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java b/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
deleted file mode 100644
index 4c3c3ab..0000000
--- a/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hive.jdbc;
-
-import java.io.OutputStream;
-import java.io.InputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.FileInputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.cli.Options;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.ql.io.RCFile.Reader;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.hive.llap.io.api.LlapProxy;
-import org.apache.hadoop.hive.llap.LlapBaseRecordReader;
-import org.apache.hadoop.hive.llap.Schema;
-
-public class LlapDump {
-
-  private static final Logger LOG = LoggerFactory.getLogger(LlapDump.class);
-
-  private static String url = "jdbc:hive2://localhost:10000/default";
-  private static String user = "hive";
-  private static String pwd = "";
-  private static String query = "select * from test";
-  private static String numSplits = "1";
-
-  public static void main(String[] args) throws Exception {
-    Options opts = createOptions();
-    CommandLine cli = new GnuParser().parse(opts, args);
-
-    if (cli.hasOption('h')) {
-      HelpFormatter formatter = new HelpFormatter();
-      formatter.printHelp("orcfiledump", opts);
-      return;
-    }
-
-    if (cli.hasOption('l')) {
-      url = cli.getOptionValue("l");
-    }
-
-    if (cli.hasOption('u')) {
-      user = cli.getOptionValue("u");
-    }
-
-    if (cli.hasOption('p')) {
-      pwd = cli.getOptionValue("p");
-    }
-
-    if (cli.hasOption('n')) {
-      numSplits = cli.getOptionValue("n");
-    }
-
-    if (cli.getArgs().length > 0) {
-      query = cli.getArgs()[0];
-    }
-
-    System.out.println("url: "+url);
-    System.out.println("user: "+user);
-    System.out.println("query: "+query);
-
-    LlapBaseInputFormat format = new LlapBaseInputFormat(url, user, pwd, query);
-    JobConf job = new JobConf();
-
-    InputSplit[] splits = format.getSplits(job, Integer.parseInt(numSplits));
-
-    if (splits.length == 0) {
-      System.out.println("No splits returned - empty scan");
-      System.out.println("Results: ");
-    } else {
-      boolean first = true;
-
-      for (InputSplit s: splits) {
-        LOG.info("Processing input split s from " + Arrays.toString(s.getLocations()));
-        RecordReader<NullWritable, Text> reader = format.getRecordReader(s, job, null);
-
-        if (reader instanceof LlapBaseRecordReader && first) {
-          Schema schema = ((LlapBaseRecordReader)reader).getSchema();
-          System.out.println(""+schema);
-        }
-
-        if (first) {
-          System.out.println("Results: ");
-          System.out.println("");
-          first = false;
-        }
-
-        Text value = reader.createValue();
-        while (reader.next(NullWritable.get(), value)) {
-          System.out.println(value);
-        }
-      }
-      System.exit(0);
-    }
-  }
-
-  static Options createOptions() {
-    Options result = new Options();
-
-    result.addOption(OptionBuilder
-        .withLongOpt("location")
-        .withDescription("HS2 url")
-        .hasArg()
-        .create('l'));
-
-    result.addOption(OptionBuilder
-        .withLongOpt("user")
-        .withDescription("user name")
-        .hasArg()
-        .create('u'));
-
-    result.addOption(OptionBuilder
-        .withLongOpt("pwd")
-        .withDescription("password")
-        .hasArg()
-        .create('p'));
-
-    result.addOption(OptionBuilder
-        .withLongOpt("num")
-        .withDescription("number of splits")
-        .hasArg()
-        .create('n'));
-
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/jdbc/src/java/org/apache/hive/jdbc/LlapInputSplit.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapInputSplit.java b/jdbc/src/java/org/apache/hive/jdbc/LlapInputSplit.java
deleted file mode 100644
index 0f4fd4e..0000000
--- a/jdbc/src/java/org/apache/hive/jdbc/LlapInputSplit.java
+++ /dev/null
@@ -1,73 +0,0 @@
-package org.apache.hive.jdbc;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
-import org.apache.hadoop.mapred.SplitLocationInfo;
-
-
-public class LlapInputSplit<V extends WritableComparable> implements InputSplitWithLocationInfo {
-  InputSplitWithLocationInfo nativeSplit;
-  String inputFormatClassName;
-
-  public LlapInputSplit() {
-  }
-
-  public LlapInputSplit(InputSplitWithLocationInfo nativeSplit, String inputFormatClassName) {
-    this.nativeSplit = nativeSplit;
-    this.inputFormatClassName = inputFormatClassName;
-  }
-
-  @Override
-  public long getLength() throws IOException {
-    return nativeSplit.getLength();
-  }
-
-  @Override
-  public String[] getLocations() throws IOException {
-    return nativeSplit.getLocations();
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeUTF(inputFormatClassName);
-    out.writeUTF(nativeSplit.getClass().getName());
-    nativeSplit.write(out);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    inputFormatClassName = in.readUTF();
-    String splitClass = in.readUTF();
-    try {
-      nativeSplit = (InputSplitWithLocationInfo)Class.forName(splitClass).newInstance();
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-    nativeSplit.readFields(in);
-  }
-
-  @Override
-  public SplitLocationInfo[] getLocationInfo() throws IOException {
-    return nativeSplit.getLocationInfo();
-  }
-
-  public InputSplit getSplit() {
-    return nativeSplit;
-  }
-
-  public InputFormat<NullWritable, V> getInputFormat() throws IOException {
-    try {
-      return (InputFormat<NullWritable, V>) Class.forName(inputFormatClassName)
-          .newInstance();
-    } catch(Exception e) {
-      throw new IOException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/jdbc/src/java/org/apache/hive/jdbc/LlapRowInputFormat.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapRowInputFormat.java b/jdbc/src/java/org/apache/hive/jdbc/LlapRowInputFormat.java
deleted file mode 100644
index 1cca66a..0000000
--- a/jdbc/src/java/org/apache/hive/jdbc/LlapRowInputFormat.java
+++ /dev/null
@@ -1,34 +0,0 @@
-package org.apache.hive.jdbc;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hive.llap.LlapBaseRecordReader;
-import org.apache.hadoop.hive.llap.LlapRowRecordReader;
-import org.apache.hadoop.hive.llap.Row;
-import org.apache.hadoop.hive.llap.Schema;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-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.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-
-public class LlapRowInputFormat implements InputFormat<NullWritable, Row> {
-  LlapBaseInputFormat<Text> baseInputFormat = new LlapBaseInputFormat<Text>();
-
-  @Override
-  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
-    return baseInputFormat.getSplits(job, numSplits);
-  }
-
-  @Override
-  public RecordReader<NullWritable, Row> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
-      throws IOException {
-    LlapInputSplit<Text> llapSplit = (LlapInputSplit<Text>) split;
-    LlapBaseRecordReader<Text> reader = (LlapBaseRecordReader<Text>) baseInputFormat.getRecordReader(llapSplit, job, reporter);
-    return new LlapRowRecordReader(job, reader.getSchema(), reader);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/llap-client/pom.xml
----------------------------------------------------------------------
diff --git a/llap-client/pom.xml b/llap-client/pom.xml
index 50c06a4..4a75bbb 100644
--- a/llap-client/pom.xml
+++ b/llap-client/pom.xml
@@ -109,6 +109,38 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-api</artifactId>
+      <version>${tez.version}</version>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commmons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-runtime-internals</artifactId>
+      <version>${tez.version}</version>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commmons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
   </dependencies>
   <build>
     <sourceDirectory>${basedir}/src/java</sourceDirectory>

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
new file mode 100644
index 0000000..7073280
--- /dev/null
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
@@ -0,0 +1,205 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.llap;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.DataInputStream;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.llap.Schema;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.JobConf;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LlapBaseRecordReader<V extends WritableComparable> implements RecordReader<NullWritable, V> {
+  private static final Logger LOG = LoggerFactory.getLogger(LlapBaseRecordReader.class);
+
+  DataInputStream din;
+  Schema schema;
+  Class<V> clazz;
+
+
+  protected Thread readerThread = null;
+  protected LinkedBlockingQueue<ReaderEvent> readerEvents = new LinkedBlockingQueue<ReaderEvent>();
+
+  public LlapBaseRecordReader(InputStream in, Schema schema, Class<V> clazz) {
+    din = new DataInputStream(in);
+    this.schema = schema;
+    this.clazz = clazz;
+    this.readerThread = Thread.currentThread();
+  }
+
+  public Schema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public void close() throws IOException {
+    din.close();
+  }
+
+  @Override
+  public long getPos() { return 0; }
+
+  @Override
+  public float getProgress() { return 0f; }
+
+  @Override
+  public NullWritable createKey() {
+    return NullWritable.get();
+  }
+
+  @Override
+  public V createValue() {
+    try {
+      return clazz.newInstance();
+    } catch (Exception e) {
+      return null;
+    }
+  }
+
+  @Override
+  public boolean next(NullWritable key, V value) throws IOException {
+    try {
+      // Need a way to know what thread to interrupt, since this is a blocking thread.
+      setReaderThread(Thread.currentThread());
+
+      value.readFields(din);
+      return true;
+    } catch (EOFException eof) {
+      // End of input. There should be a reader event available, or coming soon, so okay to be blocking call.
+      ReaderEvent event = getReaderEvent();
+      switch (event.getEventType()) {
+        case DONE:
+          break;
+        default:
+          throw new IOException("Expected reader event with done status, but got "
+              + event.getEventType() + " with message " + event.getMessage());
+      }
+      return false;
+    } catch (IOException io) {
+      if (Thread.interrupted()) {
+        // Either we were interrupted by one of:
+        // 1. handleEvent(), in which case there is a reader event waiting for us in the queue
+        // 2. Some other unrelated cause which interrupted us, in which case there may not be a reader event coming.
+        // Either way we should not try to block trying to read the reader events queue.
+        if (readerEvents.isEmpty()) {
+          // Case 2.
+          throw io;
+        } else {
+          // Case 1. Fail the reader, sending back the error we received from the reader event.
+          ReaderEvent event = getReaderEvent();
+          switch (event.getEventType()) {
+            case ERROR:
+              throw new IOException("Received reader event error: " + event.getMessage());
+            default:
+              throw new IOException("Got reader event type " + event.getEventType() + ", expected error event");
+          }
+        }
+      } else {
+        // If we weren't interrupted, just propagate the error
+        throw io;
+      }
+    }
+  }
+
+  /**
+   * Define success/error events which are passed to the reader from a different thread.
+   * The reader will check for these events on end of input and interruption of the reader thread.
+   */
+  public static class ReaderEvent {
+    public enum EventType {
+      DONE,
+      ERROR
+    }
+
+    protected final EventType eventType;
+    protected final String message;
+
+    protected ReaderEvent(EventType type, String message) {
+      this.eventType = type;
+      this.message = message;
+    }
+
+    public static ReaderEvent doneEvent() {
+      return new ReaderEvent(EventType.DONE, "");
+    }
+
+    public static ReaderEvent errorEvent(String message) {
+      return new ReaderEvent(EventType.ERROR, message);
+    }
+
+    public EventType getEventType() {
+      return eventType;
+    }
+
+    public String getMessage() {
+      return message;
+    }
+  }
+
+  public void handleEvent(ReaderEvent event) {
+    switch (event.getEventType()) {
+      case DONE:
+        // Reader will check for the event queue upon the end of the input stream - no need to interrupt.
+        readerEvents.add(event);
+        break;
+      case ERROR:
+        readerEvents.add(event);
+        if (readerThread == null) {
+          throw new RuntimeException("Reader thread is unexpectedly null, during ReaderEvent error " + event.getMessage());
+        }
+        // Reader is using a blocking socket .. interrupt it.
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Interrupting reader thread due to reader event with error " + event.getMessage());
+        }
+        getReaderThread().interrupt();
+        break;
+      default:
+        throw new RuntimeException("Unhandled ReaderEvent type " + event.getEventType() + " with message " + event.getMessage());
+    }
+  }
+
+  protected ReaderEvent getReaderEvent() {
+    try {
+      ReaderEvent event = readerEvents.take();
+      return event;
+    } catch (InterruptedException ie) {
+      throw new RuntimeException("Interrupted while getting readerEvents, not expected", ie);
+    }
+  }
+
+  protected synchronized void setReaderThread(Thread readerThread) {
+    this.readerThread = readerThread;
+  }
+
+  protected synchronized Thread getReaderThread() {
+    return readerThread;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
new file mode 100644
index 0000000..0930d60
--- /dev/null
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
@@ -0,0 +1,392 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.llap;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import com.google.common.collect.Lists;
+import com.google.protobuf.ByteString;
+
+import org.apache.commons.collections4.ListUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.llap.LlapBaseRecordReader.ReaderEvent;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
+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.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.Token;
+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.runtime.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.tez.dag.records.TezTaskAttemptID;
+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;
+
+
+public class LlapInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LlapInputFormat.class);
+
+  public LlapInputFormat() {
+  }
+
+  /*
+   * This proxy record reader has the duty of establishing a connected socket with LLAP, then fire
+   * off the work in the split to LLAP and finally return the connected socket back in an
+   * LlapRecordReader. The LlapRecordReader class reads the results from the socket.
+   */
+  @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 {
+    throw new IOException("These are not the splits you are looking for.");
+  }
+
+  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();
+    // 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());
+
+    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);
+
+
+    builder.setUsingTezAm(false);
+    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/7b9096a9/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
new file mode 100644
index 0000000..02aedfd
--- /dev/null
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.llap;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hive.llap.Schema;
+import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
+import org.apache.hadoop.mapred.SplitLocationInfo;
+import org.apache.thrift.TDeserializer;
+import org.apache.thrift.TSerializer;
+
+public class LlapInputSplit implements InputSplitWithLocationInfo {
+
+  int splitNum;
+  byte[] planBytes;
+  byte[] fragmentBytes;
+  SplitLocationInfo[] locations;
+  Schema schema;
+  String llapUser;
+
+  public LlapInputSplit() {
+  }
+
+  public LlapInputSplit(int splitNum, byte[] planBytes, byte[] fragmentBytes, SplitLocationInfo[] locations, Schema schema, String llapUser) {
+    this.planBytes = planBytes;
+    this.fragmentBytes = fragmentBytes;
+    this.locations = locations;
+    this.schema = schema;
+    this.splitNum = splitNum;
+    this.llapUser = llapUser;
+  }
+
+  public Schema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public long getLength() throws IOException {
+    return 0;
+  }
+
+  @Override
+  public String[] getLocations() throws IOException {
+    String[] locs = new String[locations.length];
+    for (int i = 0; i < locations.length; ++i) {
+      locs[i] = locations[i].getLocation();
+    }
+    return locs;
+  }
+
+  public int getSplitNum() {
+    return splitNum;
+  }
+
+  public byte[] getPlanBytes() {
+    return planBytes;
+  }
+
+  public byte[] getFragmentBytes() {
+    return fragmentBytes;
+  }
+
+
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(splitNum);
+    out.writeInt(planBytes.length);
+    out.write(planBytes);
+
+    out.writeInt(fragmentBytes.length);
+    out.write(fragmentBytes);
+
+    out.writeInt(locations.length);
+    for (int i = 0; i < locations.length; ++i) {
+      out.writeUTF(locations[i].getLocation());
+    }
+
+    schema.write(out);
+    out.writeUTF(llapUser);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    splitNum = in.readInt();
+    int length = in.readInt();
+    planBytes = new byte[length];
+    in.readFully(planBytes);
+
+    length = in.readInt();
+    fragmentBytes = new byte[length];
+    in.readFully(fragmentBytes);
+
+    length = in.readInt();
+    locations = new SplitLocationInfo[length];
+
+    for (int i = 0; i < length; ++i) {
+      locations[i] = new SplitLocationInfo(in.readUTF(), false);
+    }
+
+    schema = new Schema();
+    schema.readFields(in);
+    llapUser = in.readUTF();
+  }
+
+  @Override
+  public SplitLocationInfo[] getLocationInfo() throws IOException {
+    return locations;
+  }
+
+  public String getLlapUser() {
+    return llapUser;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
new file mode 100644
index 0000000..4e000ff
--- /dev/null
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
@@ -0,0 +1,155 @@
+package org.apache.hadoop.hive.llap;
+
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.JobConf;
+
+import org.apache.hadoop.hive.llap.Row;
+import org.apache.hadoop.hive.llap.FieldDesc;
+import org.apache.hadoop.hive.llap.Schema;
+import org.apache.hadoop.hive.llap.TypeDesc;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
+import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
+import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class LlapRowRecordReader implements RecordReader<NullWritable, Row> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LlapRowRecordReader.class);
+
+  Configuration conf;
+  RecordReader<NullWritable, Text> reader;
+  Schema schema;
+  SerDe serde;
+  final Text textData = new Text();
+
+  public LlapRowRecordReader(Configuration conf, Schema schema, RecordReader<NullWritable, Text> reader) {
+    this.conf = conf;
+    this.schema = schema;
+    this.reader = reader;
+  }
+
+  @Override
+  public void close() throws IOException {
+    reader.close();
+  }
+
+  @Override
+  public NullWritable createKey() {
+    return NullWritable.get();
+  }
+
+  @Override
+  public Row createValue() {
+    return new Row(schema);
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return 0;
+  }
+
+  @Override
+  public float getProgress() throws IOException {
+    return 0;
+  }
+
+  @Override
+  public boolean next(NullWritable key, Row value) throws IOException {
+    Preconditions.checkArgument(value != null);
+
+    if (serde == null) {
+      try {
+        serde = initSerDe(conf);
+      } catch (SerDeException err) {
+        throw new IOException(err);
+      }
+    }
+
+    boolean hasNext = reader.next(key,  textData);
+    if (hasNext) {
+      // Deserialize Text to column values, and populate the row record
+      Object rowObj;
+      try {
+        StructObjectInspector rowOI = (StructObjectInspector) serde.getObjectInspector();
+        rowObj = serde.deserialize(textData);
+        List<? extends StructField> colFields = rowOI.getAllStructFieldRefs();
+        for (int idx = 0; idx < colFields.size(); ++idx) {
+          StructField field = colFields.get(idx);
+          Object colValue = rowOI.getStructFieldData(rowObj, field);
+          Preconditions.checkState(field.getFieldObjectInspector().getCategory() == Category.PRIMITIVE,
+              "Cannot handle non-primitive column type " + field.getFieldObjectInspector().getTypeName());
+
+          PrimitiveObjectInspector poi = (PrimitiveObjectInspector) field.getFieldObjectInspector();
+          // char/varchar special cased here since the row record handles them using Text
+          switch (poi.getPrimitiveCategory()) {
+            case CHAR:
+              value.setValue(idx, ((HiveCharWritable) poi.getPrimitiveWritableObject(colValue)).getPaddedValue());
+              break;
+            case VARCHAR:
+              value.setValue(idx, ((HiveVarcharWritable) poi.getPrimitiveWritableObject(colValue)).getTextValue());
+              break;
+            default:
+              value.setValue(idx, (Writable) poi.getPrimitiveWritableObject(colValue));
+              break;
+          }
+        }
+      } catch (SerDeException err) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Error deserializing row from text: " + textData);
+        }
+        throw new IOException("Error deserializing row data", err);
+      }
+    }
+
+    return hasNext;
+  }
+
+  public Schema getSchema() {
+    return schema;
+  }
+
+  protected SerDe initSerDe(Configuration conf) throws SerDeException {
+    Properties props = new Properties();
+    StringBuffer columnsBuffer = new StringBuffer();
+    StringBuffer typesBuffer = new StringBuffer();
+    boolean isFirst = true;
+    for (FieldDesc colDesc : schema.getColumns()) {
+      if (!isFirst) {
+        columnsBuffer.append(',');
+        typesBuffer.append(',');
+      }
+      columnsBuffer.append(colDesc.getName());
+      typesBuffer.append(colDesc.getTypeDesc().toString());
+      isFirst = false;
+    }
+    String columns = columnsBuffer.toString();
+    String types = typesBuffer.toString();
+    props.put(serdeConstants.LIST_COLUMNS, columns);
+    props.put(serdeConstants.LIST_COLUMN_TYPES, types);
+    SerDe serde = new LazySimpleSerDe();
+    serde.initialize(conf, props);
+
+    return serde;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java b/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
new file mode 100644
index 0000000..83149ab
--- /dev/null
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
@@ -0,0 +1,103 @@
+package org.apache.hadoop.hive.llap;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.common.security.JobTokenIdentifier;
+import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+
+public class SubmitWorkInfo implements Writable {
+
+  private TaskSpec taskSpec;
+  private ApplicationId fakeAppId;
+  private long creationTime;
+
+  // This is used to communicate over the LlapUmbilicalProtocol. Not related to tokens used to
+  // talk to LLAP daemons itself via the securit work.
+  private Token<JobTokenIdentifier> token;
+
+  public SubmitWorkInfo(TaskSpec taskSpec, ApplicationId fakeAppId, long creationTime) {
+    this.taskSpec = taskSpec;
+    this.fakeAppId = fakeAppId;
+    this.token = createJobToken();
+    this.creationTime = creationTime;
+  }
+
+  // Empty constructor for writable etc.
+  public SubmitWorkInfo() {
+  }
+
+  public TaskSpec getTaskSpec() {
+    return taskSpec;
+  }
+
+  public ApplicationId getFakeAppId() {
+    return fakeAppId;
+  }
+
+  public String getTokenIdentifier() {
+    return fakeAppId.toString();
+  }
+
+  public Token<JobTokenIdentifier> getToken() {
+    return token;
+  }
+
+  public long getCreationTime() {
+    return creationTime;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    taskSpec.write(out);
+    out.writeLong(fakeAppId.getClusterTimestamp());
+    out.writeInt(fakeAppId.getId());
+    token.write(out);
+    out.writeLong(creationTime);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    taskSpec = new TaskSpec();
+    taskSpec.readFields(in);
+    long appIdTs = in.readLong();
+    int appIdId = in.readInt();
+    fakeAppId = ApplicationId.newInstance(appIdTs, appIdId);
+    token = new Token<>();
+    token.readFields(in);
+    creationTime = in.readLong();
+  }
+
+  public static byte[] toBytes(SubmitWorkInfo submitWorkInfo) throws IOException {
+    DataOutputBuffer dob = new DataOutputBuffer();
+    submitWorkInfo.write(dob);
+    return dob.getData();
+  }
+
+  public static SubmitWorkInfo fromBytes(byte[] submitWorkInfoBytes) throws IOException {
+    DataInputBuffer dib = new DataInputBuffer();
+    dib.reset(submitWorkInfoBytes, 0, submitWorkInfoBytes.length);
+    SubmitWorkInfo submitWorkInfo = new SubmitWorkInfo();
+    submitWorkInfo.readFields(dib);
+    return submitWorkInfo;
+  }
+
+
+  private Token<JobTokenIdentifier> createJobToken() {
+    String tokenIdentifier = fakeAppId.toString();
+    JobTokenIdentifier identifier = new JobTokenIdentifier(new Text(
+        tokenIdentifier));
+    Token<JobTokenIdentifier> sessionToken = new Token<JobTokenIdentifier>(identifier,
+        new JobTokenSecretManager());
+    sessionToken.setService(identifier.getJobId());
+    return sessionToken;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/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
new file mode 100644
index 0000000..7d06637
--- /dev/null
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
@@ -0,0 +1,415 @@
+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.SubmitWorkRequestProto;
+import org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol;
+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 {
+    int numHandlers = HiveConf.getIntVar(conf,
+        HiveConf.ConfVars.LLAP_TMP_EXT_CLIENT_NUM_SERVER_HANDLERS);
+    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. This should always have the usingTezAm flag disabled
+   * @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(
+        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.";
+                LOG.info(msg);
+                if (responder != null) {
+                  Throwable err = new RuntimeException(msg);
+                  responder.submissionFailed(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), err);
+                }
+                return;
+              }
+            }
+          }
+
+          @Override
+          public void indicateError(Throwable t) {
+            String msg = "Failed to submit: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString();
+            LOG.error(msg, t);
+            Throwable err = new RuntimeException(msg, t);
+            responder.submissionFailed(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), 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/7b9096a9/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java b/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
new file mode 100644
index 0000000..dbd591a
--- /dev/null
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
@@ -0,0 +1,57 @@
+package org.apache.hadoop.hive.llap.tezplugins.helpers;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.Token;
+import org.apache.tez.common.security.JobTokenIdentifier;
+import org.apache.tez.common.security.JobTokenSecretManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LlapTaskUmbilicalServer {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LlapTaskUmbilicalServer.class);
+
+  protected volatile Server server;
+  private final InetSocketAddress address;
+  private final AtomicBoolean started = new AtomicBoolean(true);
+
+  public LlapTaskUmbilicalServer(Configuration conf, LlapTaskUmbilicalProtocol umbilical, int numHandlers, String tokenIdentifier, Token<JobTokenIdentifier> token) throws
+      IOException {
+    JobTokenSecretManager jobTokenSecretManager =
+        new JobTokenSecretManager();
+    jobTokenSecretManager.addTokenForJob(tokenIdentifier, token);
+
+    server = new RPC.Builder(conf)
+        .setProtocol(LlapTaskUmbilicalProtocol.class)
+        .setBindAddress("0.0.0.0")
+        .setPort(0)
+        .setInstance(umbilical)
+        .setNumHandlers(numHandlers)
+        .setSecretManager(jobTokenSecretManager).build();
+
+    server.start();
+    this.address = NetUtils.getConnectAddress(server);
+    LOG.info(
+        "Started TaskUmbilicalServer: " + umbilical.getClass().getName() + " at address: " + address +
+            " with numHandlers=" + numHandlers);
+  }
+
+  public InetSocketAddress getAddress() {
+    return this.address;
+  }
+
+  public void shutdownServer() {
+    if (started.get()) { // Primarily to avoid multiple shutdowns.
+      started.set(false);
+      server.stop();
+    }
+  }
+}


[39/39] hive git commit: Merge branch 'master' into llap

Posted by jd...@apache.org.
Merge branch 'master' into llap


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f089f2e6
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f089f2e6
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f089f2e6

Branch: refs/heads/master
Commit: f089f2e64241592ecf8144d044bec8a0659ff422
Parents: 89ec219 3f07bfc
Author: Jason Dere <jd...@hortonworks.com>
Authored: Fri May 6 10:14:21 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Fri May 6 10:14:21 2016 -0700

----------------------------------------------------------------------
 beeline/src/main/resources/BeeLine.properties   |   9 +-
 cli/pom.xml                                     |   6 +
 common/pom.xml                                  |   6 +
 .../org/apache/hadoop/hive/common/LogUtils.java |  35 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   8 +
 .../src/main/resources/hive-log4j2.properties   |   2 +-
 .../hadoop/hive/conf/TestHiveAsyncLogging.java  |  49 ++
 data/conf/hive-log4j2.properties                |   2 +-
 hcatalog/core/pom.xml                           |   6 +
 .../hive/metastore/TestHiveMetaStoreTxns.java   |   2 +-
 llap-server/bin/llapDaemon.sh                   |   2 +-
 .../hadoop/hive/llap/cli/LlapServiceDriver.java |  21 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |  10 +-
 ...doop-metrics2-llapdaemon.properties.template |  50 ++
 ...trics2-llaptaskscheduler.properties.template |  50 ++
 .../hadoop-metrics2.properties.template         |  50 --
 .../main/resources/llap-cli-log4j2.properties   |   2 +-
 .../resources/llap-daemon-log4j2.properties     |   4 +-
 llap-server/src/main/resources/package.py       |   6 +-
 .../resources/llap-daemon-log4j2.properties     |   4 +-
 .../tezplugins/LlapTaskSchedulerService.java    |   2 +-
 .../metrics/LlapTaskSchedulerMetrics.java       |   6 +-
 metastore/pom.xml                               |   6 +
 .../upgrade/derby/035-HIVE-13395.derby.sql      |  11 +
 .../upgrade/derby/hive-schema-2.1.0.derby.sql   |   2 +-
 .../derby/hive-txn-schema-1.3.0.derby.sql       |  11 +-
 .../derby/hive-txn-schema-2.1.0.derby.sql       | 130 ++++
 .../derby/upgrade-1.2.0-to-1.3.0.derby.sql      |   1 +
 .../derby/upgrade-2.0.0-to-2.1.0.derby.sql      |   1 +
 .../upgrade/mssql/020-HIVE-13395.mssql.sql      |   9 +
 .../upgrade/mssql/hive-schema-1.3.0.mssql.sql   |  12 +-
 .../upgrade/mssql/hive-schema-2.1.0.mssql.sql   |  12 +-
 .../mssql/upgrade-1.2.0-to-1.3.0.mssql.sql      |   1 +
 .../mssql/upgrade-2.0.0-to-2.1.0.mssql.sql      |   1 +
 .../upgrade/mysql/035-HIVE-13395.mysql.sql      |  10 +
 .../upgrade/mysql/hive-schema-2.1.0.mysql.sql   |   2 +-
 .../mysql/hive-txn-schema-1.3.0.mysql.sql       |  10 +
 .../mysql/hive-txn-schema-2.1.0.mysql.sql       | 131 ++++
 .../mysql/upgrade-1.2.0-to-1.3.0.mysql.sql      |   1 +
 .../mysql/upgrade-2.0.0-to-2.1.0.mysql.sql      |   1 +
 .../upgrade/oracle/035-HIVE-13395.oracle.sql    |  10 +
 .../upgrade/oracle/hive-schema-2.1.0.oracle.sql |   2 +-
 .../oracle/hive-txn-schema-1.3.0.oracle.sql     |  12 +-
 .../oracle/hive-txn-schema-2.1.0.oracle.sql     | 129 ++++
 .../oracle/upgrade-1.2.0-to-1.3.0.oracle.sql    |   1 +
 .../oracle/upgrade-2.0.0-to-2.1.0.oracle.sql    |   1 +
 .../postgres/034-HIVE-13395.postgres.sql        |  10 +
 .../postgres/hive-schema-2.1.0.postgres.sql     |   2 +-
 .../postgres/hive-txn-schema-1.3.0.postgres.sql |  11 +-
 .../postgres/hive-txn-schema-2.1.0.postgres.sql | 129 ++++
 .../upgrade-1.2.0-to-1.3.0.postgres.sql         |   1 +
 .../upgrade-2.0.0-to-2.1.0.postgres.sql         |   1 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   1 +
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    | 130 ++--
 .../hadoop/hive/metastore/txn/TxnHandler.java   | 466 +++++++++++---
 .../hadoop/hive/metastore/txn/TxnStore.java     |   8 +-
 .../hadoop/hive/metastore/txn/TxnUtils.java     |   2 +
 .../metastore/txn/TestCompactionTxnHandler.java |   6 +-
 .../hive/metastore/txn/TestTxnHandler.java      |  29 +-
 pom.xml                                         |   2 +
 ql/pom.xml                                      |   6 +
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  11 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |   2 +-
 .../hadoop/hive/ql/exec/OperatorUtils.java      |   2 +-
 .../org/apache/hadoop/hive/ql/exec/Task.java    |  21 +-
 .../apache/hadoop/hive/ql/exec/TaskResult.java  |   7 +-
 .../apache/hadoop/hive/ql/exec/TaskRunner.java  |   5 +-
 .../hive/ql/exec/mr/HadoopJobExecHelper.java    |   1 +
 .../hadoop/hive/ql/exec/mr/JobDebugger.java     |  18 +-
 .../hadoop/hive/ql/lockmgr/DbLockManager.java   |   5 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |  27 +-
 .../calcite/translator/JoinTypeCheckCtx.java    |   2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  17 +-
 .../hadoop/hive/ql/parse/TypeCheckCtx.java      |  19 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |  26 +
 .../hadoop/hive/ql/txn/AcidWriteSetService.java |  61 ++
 .../txn/compactor/HouseKeeperServiceBase.java   |   2 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java |   2 +-
 .../hadoop/hive/ql/txn/compactor/Worker.java    |   2 +-
 .../main/resources/hive-exec-log4j2.properties  |   2 +-
 .../resources/tez-container-log4j2.properties   |   2 +-
 .../apache/hadoop/hive/ql/TestTxnCommands2.java |   2 +-
 .../apache/hadoop/hive/ql/io/TestAcidUtils.java |  20 +
 .../hive/ql/lockmgr/TestDbTxnManager.java       |   7 +
 .../hive/ql/lockmgr/TestDbTxnManager2.java      | 610 ++++++++++++++++++-
 .../hive/ql/txn/compactor/TestCleaner.java      |   4 +
 .../queries/clientpositive/constantPropWhen.q   |   2 +
 .../hive/ptest/execution/ExecutionPhase.java    |   2 +
 .../hive/ptest/execution/HostExecutor.java      |  48 +-
 .../hive/ptest/execution/LocalCommand.java      |  31 +-
 .../apache/hive/ptest/execution/PrepPhase.java  |   1 +
 .../apache/hive/ptest/execution/conf/Host.java  |   3 +
 92 files changed, 2294 insertions(+), 313 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f089f2e6/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f089f2e6/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f089f2e6/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f089f2e6/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f089f2e6/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------


[11/39] hive git commit: HIVE-13305: LlapInputFormat should get LlapOutputFormatService port from the LLAP service registry

Posted by jd...@apache.org.
HIVE-13305: LlapInputFormat should get LlapOutputFormatService port from the LLAP service registry


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/25140659
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/25140659
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/25140659

Branch: refs/heads/master
Commit: 25140659c578e3e01a3ce36ba5108a38303dc843
Parents: 2945c3b
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu Mar 17 16:12:34 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu Mar 17 16:12:34 2016 -0700

----------------------------------------------------------------------
 .../hive/llap/registry/ServiceInstance.java     |  7 ++
 .../registry/impl/LlapFixedRegistryImpl.java    |  7 ++
 .../impl/LlapZookeeperRegistryImpl.java         | 16 ++++
 .../hadoop/hive/llap/LlapInputFormat.java       | 78 ++++++++++++++++++--
 4 files changed, 102 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/25140659/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstance.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstance.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstance.java
index 2bd860a..a504146 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstance.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstance.java
@@ -54,6 +54,13 @@ public interface ServiceInstance {
   public int getShufflePort();
 
   /**
+   * OutputFormat endpoint for service instance
+   *
+   * @return
+   */
+  public int getOutputFormatPort();
+
+  /**
    * Return the last known state (without refreshing)
    * 
    * @return

http://git-wip-us.apache.org/repos/asf/hive/blob/25140659/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
index 8cace8f..33ab591 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
@@ -52,6 +52,7 @@ public class LlapFixedRegistryImpl implements ServiceRegistry {
   private final int port;
   private final int shuffle;
   private final int mngPort;
+  private final int outputFormatPort;
   private final String[] hosts;
   private final int memory;
   private final int vcores;
@@ -65,6 +66,7 @@ public class LlapFixedRegistryImpl implements ServiceRegistry {
     this.shuffle = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_YARN_SHUFFLE_PORT);
     this.resolveHosts = conf.getBoolean(FIXED_REGISTRY_RESOLVE_HOST_NAMES, true);
     this.mngPort = HiveConf.getIntVar(conf, ConfVars.LLAP_MANAGEMENT_RPC_PORT);
+    this.outputFormatPort = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
 
     for (Map.Entry<String, String> kv : conf) {
       if (kv.getKey().startsWith(HiveConf.PREFIX_LLAP)
@@ -151,6 +153,11 @@ public class LlapFixedRegistryImpl implements ServiceRegistry {
     }
 
     @Override
+    public int getOutputFormatPort() {
+      return LlapFixedRegistryImpl.this.outputFormatPort;
+    }
+
+    @Override
     public boolean isAlive() {
       return true;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/25140659/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
index ab9fa39..c440e1e 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
@@ -90,6 +90,7 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
   private static final String IPC_MNG = "llapmng";
   private static final String IPC_SHUFFLE = "shuffle";
   private static final String IPC_LLAP = "llap";
+  private static final String IPC_OUTPUTFORMAT = "llapoutputformat";
   private final static String ROOT_NAMESPACE = "llap";
 
   private final Configuration conf;
@@ -241,6 +242,11 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
         HiveConf.getIntVar(conf, ConfVars.LLAP_MANAGEMENT_RPC_PORT)));
   }
 
+  public Endpoint getOutputFormatEndpoint() {
+    return RegistryTypeUtils.ipcEndpoint(IPC_OUTPUTFORMAT, new InetSocketAddress(hostname,
+        HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT)));
+  }
+
   @Override
   public void register() throws IOException {
     ServiceRecord srv = new ServiceRecord();
@@ -310,6 +316,7 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
     private final int rpcPort;
     private final int mngPort;
     private final int shufflePort;
+    private final int outputFormatPort;
 
     public DynamicServiceInstance(ServiceRecord srv) throws IOException {
       this.srv = srv;
@@ -317,6 +324,7 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
       final Endpoint shuffle = srv.getInternalEndpoint(IPC_SHUFFLE);
       final Endpoint rpc = srv.getInternalEndpoint(IPC_LLAP);
       final Endpoint mng = srv.getInternalEndpoint(IPC_MNG);
+      final Endpoint outputFormat = srv.getInternalEndpoint(IPC_OUTPUTFORMAT);
 
       this.host =
           RegistryTypeUtils.getAddressField(rpc.addresses.get(0),
@@ -330,6 +338,9 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
       this.shufflePort =
           Integer.valueOf(RegistryTypeUtils.getAddressField(shuffle.addresses.get(0),
               AddressTypes.ADDRESS_PORT_FIELD));
+      this.outputFormatPort =
+          Integer.valueOf(RegistryTypeUtils.getAddressField(outputFormat.addresses.get(0),
+              AddressTypes.ADDRESS_PORT_FIELD));
     }
 
     @Override
@@ -386,6 +397,11 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
       return mngPort;
     }
 
+    @Override
+    public int getOutputFormatPort() {
+      return outputFormatPort;
+    }
+
     // Relying on the identity hashCode and equality, since refreshing instances retains the old copy
     // of an already known instance.
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/25140659/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
index b32d662..847c74f 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
@@ -17,10 +17,12 @@
 package org.apache.hadoop.hive.llap;
 
 import java.io.IOException;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.nio.ByteBuffer;
 import java.util.List;
+import java.util.Set;
 
 import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
@@ -28,6 +30,9 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
 import org.apache.hadoop.hive.llap.ext.LlapTaskUmbilicalExternalClient;
+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;
@@ -72,8 +77,12 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     LlapInputSplit llapSplit = (LlapInputSplit) split;
     SubmitWorkInfo submitWorkInfo = SubmitWorkInfo.fromBytes(llapSplit.getPlanBytes());
 
-    // TODO HACK: Spark is built with Hive-1.2.1, does not have access to HiveConf.ConfVars.LLAP_DAEMON_RPC_PORT
-    int llapSubmitPort = job.getInt("hive.llap.daemon.rpc.port", 15001);
+    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());
 
     LlapTaskUmbilicalExternalClient llapClient =
       new LlapTaskUmbilicalExternalClient(job, submitWorkInfo.getTokenIdentifier(),
@@ -92,16 +101,13 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     List<TezEvent> tezEventList = Lists.newArrayList();
     tezEventList.add(tezEvent);
 
-    // this is just the portion that sets up the io to receive data
-    String host = split.getLocations()[0];
-
     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,
-        conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT));
+        serviceInstance.getOutputFormatPort());
 
     LOG.debug("Socket connected");
 
@@ -119,6 +125,66 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     throw new IOException("These are not the splits you are looking for.");
   }
 
+  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,


[04/39] hive git commit: HIVE-13152: JDBC split refactoring and handle some edge cases (Vikram Dixit K, Gunther Hagleitner, Siddharth Seth, Jason Dere)

Posted by jd...@apache.org.
HIVE-13152: JDBC split refactoring and handle some edge cases (Vikram Dixit K, Gunther Hagleitner, Siddharth Seth, Jason Dere)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7c4b3ffd
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7c4b3ffd
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7c4b3ffd

Branch: refs/heads/master
Commit: 7c4b3ffdf95c55c7717983a56229510577c9006a
Parents: 2e042cc
Author: vikram <vi...@hortonworks.com>
Authored: Wed Feb 24 17:25:21 2016 -0800
Committer: vikram <vi...@hortonworks.com>
Committed: Wed Feb 24 17:25:21 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  2 +-
 .../hadoop/hive/jdbc/TestLlapInputSplit.java    | 99 ++++++++++++++++++++
 .../src/java/org/apache/hive/jdbc/LlapDump.java | 39 +++++---
 .../org/apache/hive/jdbc/LlapInputFormat.java   | 62 +-----------
 .../org/apache/hive/jdbc/LlapInputSplit.java    | 73 +++++++++++++++
 .../hadoop/hive/llap/LlapInputFormat.java       |  3 +-
 .../hadoop/hive/llap/LlapStorageHandler.java    | 34 +++++++
 .../apache/hadoop/hive/llap/LlapInputSplit.java | 11 ---
 .../apache/hadoop/hive/ql/plan/PlanUtils.java   |  7 ++
 .../ql/udf/generic/GenericUDFGetSplits.java     |  8 +-
 10 files changed, 251 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7c4b3ffd/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 6a22890..6e87567 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -988,7 +988,7 @@ public class HiveConf extends Configuration {
   "Default file format for CREATE TABLE statement applied to managed tables only. External tables will be \n" +
   "created with format specified by hive.default.fileformat. Leaving this null will result in using hive.default.fileformat \n" +
   "for all tables."),
-    HIVEQUERYRESULTFILEFORMAT("hive.query.result.fileformat", "SequenceFile", new StringSet("TextFile", "SequenceFile", "RCfile"),
+    HIVEQUERYRESULTFILEFORMAT("hive.query.result.fileformat", "SequenceFile", new StringSet("TextFile", "SequenceFile", "RCfile", "Llap"),
         "Default file format for storing result of the query."),
     HIVECHECKFILEFORMAT("hive.fileformat.check", true, "Whether to check file format or not when loading data files"),
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7c4b3ffd/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
new file mode 100644
index 0000000..b8b69a7
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
@@ -0,0 +1,99 @@
+package org.apache.hive.jdbc;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+
+import org.apache.hadoop.io.Text;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.mapred.SplitLocationInfo;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import static org.junit.Assert.*;
+
+public class TestLlapInputSplit {
+
+  @Test
+  public void testWritable() throws Exception {
+    int splitNum = 88;
+    byte[] planBytes = "0123456789987654321".getBytes();
+    byte[] fragmentBytes = "abcdefghijklmnopqrstuvwxyz".getBytes();
+    SplitLocationInfo[] locations = {
+        new SplitLocationInfo("location1", false),
+        new SplitLocationInfo("location2", false),
+    };
+    ArrayList<FieldSchema> fields = new ArrayList<FieldSchema>();
+    fields.add(new FieldSchema("col1", "string", "comment1"));
+    fields.add(new FieldSchema("col2", "int", "comment2"));
+    HashMap<String, String> properties = new HashMap<String, String>();
+    properties.put("key1", "val1");
+    Schema schema = new Schema(
+        fields,
+        properties);
+
+    org.apache.hadoop.hive.llap.LlapInputSplit split1 = new org.apache.hadoop.hive.llap.LlapInputSplit(
+        splitNum,
+        planBytes,
+        fragmentBytes,
+        locations,
+        schema);
+    ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream();
+    DataOutputStream dataOut = new DataOutputStream(byteOutStream);
+    split1.write(dataOut);
+    ByteArrayInputStream byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray());
+    DataInputStream dataIn = new DataInputStream(byteInStream);
+    org.apache.hadoop.hive.llap.LlapInputSplit split2 = new org.apache.hadoop.hive.llap.LlapInputSplit();
+    split2.readFields(dataIn);
+
+    // Did we read all the data?
+    assertEquals(0, byteInStream.available());
+
+    checkLlapSplits(split1, split2);
+
+    // Try JDBC LlapInputSplits
+    org.apache.hive.jdbc.LlapInputSplit<Text> jdbcSplit1 =
+        new org.apache.hive.jdbc.LlapInputSplit<Text>(split1, "org.apache.hadoop.hive.llap.LlapInputFormat");
+    byteOutStream.reset();
+    jdbcSplit1.write(dataOut);
+    byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray());
+    dataIn = new DataInputStream(byteInStream);
+    org.apache.hive.jdbc.LlapInputSplit<Text> jdbcSplit2 = new org.apache.hive.jdbc.LlapInputSplit<Text>();
+    jdbcSplit2.readFields(dataIn);
+
+    assertEquals(0, byteInStream.available());
+
+    checkLlapSplits(
+        (org.apache.hadoop.hive.llap.LlapInputSplit) jdbcSplit1.getSplit(),
+        (org.apache.hadoop.hive.llap.LlapInputSplit) jdbcSplit2.getSplit());
+    assertEquals(jdbcSplit1.getInputFormat().getClass(), jdbcSplit2.getInputFormat().getClass());
+  }
+
+  static void checkLlapSplits(
+      org.apache.hadoop.hive.llap.LlapInputSplit split1,
+      org.apache.hadoop.hive.llap.LlapInputSplit split2) throws Exception {
+
+    assertEquals(split1.getSplitNum(), split2.getSplitNum());
+    assertArrayEquals(split1.getPlanBytes(), split2.getPlanBytes());
+    assertArrayEquals(split1.getFragmentBytes(), split2.getFragmentBytes());
+    SplitLocationInfo[] locationInfo1 = split1.getLocationInfo();
+    SplitLocationInfo[] locationInfo2 = split2.getLocationInfo();
+    for (int idx = 0; idx < locationInfo1.length; ++idx) {
+      assertEquals(locationInfo1[idx].getLocation(), locationInfo2[idx].getLocation());
+      assertEquals(locationInfo1[idx].isInMemory(), locationInfo2[idx].isInMemory());
+      assertEquals(locationInfo1[idx].isOnDisk(), locationInfo2[idx].isOnDisk());
+    }
+    assertArrayEquals(split1.getLocations(), split2.getLocations());
+    assertEquals(split1.getSchema(), split2.getSchema());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7c4b3ffd/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java b/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
index b0c0253..a807f6c 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
@@ -94,19 +94,34 @@ public class LlapDump {
 
     LlapInputFormat format = new LlapInputFormat(url, user, pwd, query);
     JobConf job = new JobConf();
-    InputSplit[] splits = format.getSplits(job, 1);
-    RecordReader<NullWritable, Text> reader = format.getRecordReader(splits[0], job, null);
 
-    if (reader instanceof LlapRecordReader) {
-      Schema schema = ((LlapRecordReader)reader).getSchema();
-      System.out.println(""+schema);
-    }
-    System.out.println("Results: ");
-    System.out.println("");
+    InputSplit[] splits = format.getSplits(job, 1);
 
-    Text value = reader.createValue();
-    while (reader.next(NullWritable.get(), value)) {
-      System.out.println(value);
+    if (splits.length == 0) {
+      System.out.println("No splits returned - empty scan");
+      System.out.println("Results: ");
+    } else {
+      boolean first = true;
+
+      for (InputSplit s: splits) {
+        RecordReader<NullWritable, Text> reader = format.getRecordReader(s, job, null);
+
+        if (reader instanceof LlapRecordReader && first) {
+          Schema schema = ((LlapRecordReader)reader).getSchema();
+          System.out.println(""+schema);
+        }
+
+        if (first) {
+          System.out.println("Results: ");
+          System.out.println("");
+          first = false;
+        }
+
+        Text value = reader.createValue();
+        while (reader.next(NullWritable.get(), value)) {
+          System.out.println(value);
+        }
+      }
     }
   }
 
@@ -116,7 +131,7 @@ public class LlapDump {
     result.addOption(OptionBuilder
         .withLongOpt("location")
         .withDescription("HS2 url")
-	.hasArg()
+        .hasArg()
         .create('l'));
 
     result.addOption(OptionBuilder

http://git-wip-us.apache.org/repos/asf/hive/blob/7c4b3ffd/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java b/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
index e662414..5af2175 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
@@ -76,63 +76,11 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
 
   public LlapInputFormat() {}
 
-  public class LlapInputSplit implements InputSplitWithLocationInfo {
-    InputSplitWithLocationInfo nativeSplit;
-    String inputFormatClassName;
-
-    @Override
-    public long getLength() throws IOException {
-      return nativeSplit.getLength();
-    }
-
-    @Override
-    public String[] getLocations() throws IOException {
-      return nativeSplit.getLocations();
-    }
-
-    @Override
-    public void write(DataOutput out) throws IOException {
-      out.writeUTF(inputFormatClassName);
-      out.writeUTF(nativeSplit.getClass().toString());
-      nativeSplit.write(out);
-    }
-
-    @Override
-    public void readFields(DataInput in) throws IOException {
-      inputFormatClassName = in.readUTF();
-      String splitClass = in.readUTF();
-      try {
-        nativeSplit = (InputSplitWithLocationInfo)Class.forName(splitClass).newInstance();
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-      nativeSplit.readFields(in);
-    }
-
-    @Override
-    public SplitLocationInfo[] getLocationInfo() throws IOException {
-      return nativeSplit.getLocationInfo();
-    }
-
-    public InputSplit getSplit() {
-      return nativeSplit;
-    }
-
-    public InputFormat<NullWritable, V> getInputFormat() {
-      try {
-        return (InputFormat<NullWritable, V>) Class.forName(inputFormatClassName)
-            .newInstance();
-      } catch(Exception e) {
-        return null;
-      }
-    }
-  }
 
   @Override
   public RecordReader<NullWritable, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
-    try {
-      return ((InputFormat)Class.forName("org.apache.hadoop.hive.llap.LlapInputFormat").newInstance()).getRecordReader(split, job, reporter);
-    } catch (Exception e) { throw new IOException(e); }
+    LlapInputSplit llapSplit = (LlapInputSplit) split;
+    return llapSplit.getInputFormat().getRecordReader(llapSplit.getSplit(), job, reporter);
   }
 
   @Override
@@ -162,9 +110,9 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
       while (res.next()) {
         // deserialize split
         DataInput in = new DataInputStream(res.getBinaryStream(3));
-        InputSplit is = (InputSplitWithLocationInfo)Class.forName(res.getString(2)).newInstance(); // todo setAccessible on ctor
+        InputSplitWithLocationInfo is = (InputSplitWithLocationInfo)Class.forName(res.getString(2)).newInstance();
         is.readFields(in);
-        ins.add(is);
+        ins.add(new LlapInputSplit(is, res.getString(1)));
       }
 
       res.close();
@@ -172,7 +120,7 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     } catch (Exception e) {
       throw new IOException(e);
     }
-    return ins.toArray(new InputSplit[ins.size()]); // todo wrap input split with format
+    return ins.toArray(new InputSplit[ins.size()]);
   }
 
   public void close() {

http://git-wip-us.apache.org/repos/asf/hive/blob/7c4b3ffd/jdbc/src/java/org/apache/hive/jdbc/LlapInputSplit.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapInputSplit.java b/jdbc/src/java/org/apache/hive/jdbc/LlapInputSplit.java
new file mode 100644
index 0000000..0f4fd4e
--- /dev/null
+++ b/jdbc/src/java/org/apache/hive/jdbc/LlapInputSplit.java
@@ -0,0 +1,73 @@
+package org.apache.hive.jdbc;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
+import org.apache.hadoop.mapred.SplitLocationInfo;
+
+
+public class LlapInputSplit<V extends WritableComparable> implements InputSplitWithLocationInfo {
+  InputSplitWithLocationInfo nativeSplit;
+  String inputFormatClassName;
+
+  public LlapInputSplit() {
+  }
+
+  public LlapInputSplit(InputSplitWithLocationInfo nativeSplit, String inputFormatClassName) {
+    this.nativeSplit = nativeSplit;
+    this.inputFormatClassName = inputFormatClassName;
+  }
+
+  @Override
+  public long getLength() throws IOException {
+    return nativeSplit.getLength();
+  }
+
+  @Override
+  public String[] getLocations() throws IOException {
+    return nativeSplit.getLocations();
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeUTF(inputFormatClassName);
+    out.writeUTF(nativeSplit.getClass().getName());
+    nativeSplit.write(out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    inputFormatClassName = in.readUTF();
+    String splitClass = in.readUTF();
+    try {
+      nativeSplit = (InputSplitWithLocationInfo)Class.forName(splitClass).newInstance();
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+    nativeSplit.readFields(in);
+  }
+
+  @Override
+  public SplitLocationInfo[] getLocationInfo() throws IOException {
+    return nativeSplit.getLocationInfo();
+  }
+
+  public InputSplit getSplit() {
+    return nativeSplit;
+  }
+
+  public InputFormat<NullWritable, V> getInputFormat() throws IOException {
+    try {
+      return (InputFormat<NullWritable, V>) Class.forName(inputFormatClassName)
+          .newInstance();
+    } catch(Exception e) {
+      throw new IOException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7c4b3ffd/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
index cf13c1e..d8066d5 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
@@ -72,7 +72,8 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     LlapInputSplit llapSplit = (LlapInputSplit) split;
     SubmitWorkInfo submitWorkInfo = SubmitWorkInfo.fromBytes(llapSplit.getPlanBytes());
 
-    int llapSubmitPort = HiveConf.getIntVar(job, HiveConf.ConfVars.LLAP_DAEMON_RPC_PORT);
+    // TODO HACK: Spark is built with Hive-1.2.1, does not have access to HiveConf.ConfVars.LLAP_DAEMON_RPC_PORT
+    int llapSubmitPort = job.getInt("hive.llap.daemon.rpc.port", 15001);
 
     LOG.info("ZZZ: DBG: Starting LlapTaskUmbilicalExternalClient");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7c4b3ffd/llap-server/src/java/org/apache/hadoop/hive/llap/LlapStorageHandler.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapStorageHandler.java b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapStorageHandler.java
new file mode 100644
index 0000000..c001cc1
--- /dev/null
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapStorageHandler.java
@@ -0,0 +1,34 @@
+/*
+ * 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 org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.OutputFormat;
+
+public class LlapStorageHandler extends DefaultStorageHandler {
+  @Override
+  public Class<? extends InputFormat> getInputFormatClass() {
+    throw new RuntimeException("Should not be called.");
+  }
+
+  @Override
+  public Class<? extends OutputFormat> getOutputFormatClass() {
+    return LlapOutputFormat.class;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7c4b3ffd/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
index d26a579..2ac0ccd 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
@@ -98,10 +98,6 @@ public class LlapInputSplit implements InputSplitWithLocationInfo {
       byte[] serialzied = serializer.serialize(schema);
       out.writeInt(serialzied.length);
       out.write(serialzied);
-//      AutoExpandingBufferWriteTransport transport = new AutoExpandingBufferWriteTransport(1024, 2d);
-//      TProtocol protocol = new TBinaryProtocol(transport);
-//      schema.write(protocol);
-//      binarySchema = transport.getBuf().array();
     } catch (Exception e) {
       throw new IOException(e);
     }
@@ -135,13 +131,6 @@ public class LlapInputSplit implements InputSplitWithLocationInfo {
       TDeserializer tDeserializer = new TDeserializer();
       schema = new Schema();
       tDeserializer.deserialize(schema, schemaBytes);
-//      AutoExpandingBufferReadTransport transport = new AutoExpandingBufferReadTransport(length, 2d);
-//      AutoExpandingBuffer buf = transport.getBuf();
-//      in.readFully(buf.array(), 0, length);
-//
-//      TProtocol protocol = new TBinaryProtocol(transport);
-//      schema = new Schema();
-//      schema.read(protocol);
     } catch (Exception e) {
       throw new IOException(e);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/7c4b3ffd/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
index 04d26f3..35bca87 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
 import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
 import org.apache.hadoop.hive.ql.io.RCFileOutputFormat;
+import org.apache.hadoop.hive.llap.LlapOutputFormat;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler;
@@ -270,6 +271,12 @@ public final class PlanUtils {
       inputFormat = RCFileInputFormat.class;
       outputFormat = RCFileOutputFormat.class;
       assert serdeClass == ColumnarSerDe.class;
+    } else if ("Llap".equalsIgnoreCase(fileFormat)) {
+      inputFormat = TextInputFormat.class;
+      outputFormat = LlapOutputFormat.class;
+      properties.setProperty(
+          org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE,
+          "org.apache.hadoop.hive.llap.LlapStorageHandler");
     } else { // use TextFile by default
       inputFormat = TextInputFormat.class;
       outputFormat = IgnoreKeyTextOutputFormat.class;

http://git-wip-us.apache.org/repos/asf/hive/blob/7c4b3ffd/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
index 9fa4aa8..f69dea3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
@@ -122,13 +122,13 @@ public class GenericUDFGetSplits extends GenericUDF {
     try {
       if (SessionState.get() != null && SessionState.get().getConf() != null) {
         HiveConf conf = SessionState.get().getConf();
-        jc = new JobConf(conf);
+        jc = DagUtils.getInstance().createConfiguration(conf);
         db = Hive.get(conf);
       } else {
         jc = MapredContext.get().getJobConf();
         db = Hive.get();
       }
-    } catch(HiveException e) {
+    } catch(Exception e) {
       LOG.error("Failed to initialize: ",e);
       throw new UDFArgumentException(e);
     }
@@ -189,7 +189,7 @@ public class GenericUDFGetSplits extends GenericUDF {
     try {
       LOG.info("setting fetch.task.conversion to none and query file format to \""+LlapOutputFormat.class.getName()+"\"");
       HiveConf.setVar(conf, ConfVars.HIVEFETCHTASKCONVERSION, "none");
-      HiveConf.setVar(conf, HiveConf.ConfVars.HIVEQUERYRESULTFILEFORMAT, LlapOutputFormat.class.getName());
+      HiveConf.setVar(conf, HiveConf.ConfVars.HIVEQUERYRESULTFILEFORMAT, "Llap");
 
       cpr = driver.compileAndRespond(query);
       if(cpr.getResponseCode() != 0) {
@@ -204,8 +204,6 @@ public class GenericUDFGetSplits extends GenericUDF {
         throw new HiveException("Was expecting a single TezTask.");
       }
 
-      Path data = null;
-
       TezWork tezWork = ((TezTask)roots.get(0)).getWork();
 
       if (tezWork.getAllWork().size() != 1) {


[02/39] hive git commit: HIVE-13138. Add client to communicate with interface, initial split setup. (Siddharth Seth and Vikram Dixit K)

Posted by jd...@apache.org.
HIVE-13138. Add client to communicate with interface, initial split
setup. (Siddharth Seth and Vikram Dixit K)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f272acea
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f272acea
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f272acea

Branch: refs/heads/master
Commit: f272aceaf7da77f9d87f5be42bb1520181035c2c
Parents: bf83407
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Feb 23 23:55:46 2016 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Feb 23 23:55:46 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   3 +
 .../org/apache/hive/jdbc/LlapInputFormat.java   |  10 +
 .../daemon/rpc/LlapDaemonProtocolProtos.java    | 159 ++++++++++---
 .../src/protobuf/LlapDaemonProtocol.proto       |   8 +
 .../hive/llap/daemon/impl/LlapDaemon.java       |   3 +
 .../llap/daemon/impl/TaskRunnerCallable.java    |   5 +-
 .../ext/LlapTaskUmbilicalExternalClient.java    | 197 ++++++++++++++++
 .../helpers/LlapTaskUmbilicalServer.java        |  57 +++++
 .../hadoop/hive/llap/LlapInputFormat.java       | 146 +++---------
 .../apache/hadoop/hive/llap/LlapInputSplit.java |  80 ++++---
 .../apache/hadoop/hive/llap/SubmitWorkInfo.java |  65 ++++++
 .../hive/ql/exec/tez/HiveSplitGenerator.java    |  49 +++-
 .../hive/ql/exec/tez/MapRecordProcessor.java    |   2 +
 .../hive/ql/parse/TypeCheckProcFactory.java     |   3 +
 .../ql/udf/generic/GenericUDFGetSplits.java     | 224 +++++++++++++++++--
 .../org/apache/tez/dag/api/TaskSpecBuilder.java |  45 ++++
 16 files changed, 837 insertions(+), 219 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 7fbcbba..6a22890 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2687,6 +2687,9 @@ public class HiveConf extends Configuration {
     LLAP_DAEMON_OUTPUT_SERVICE_PORT("hive.llap.daemon.output.service.port", 15003,
         "LLAP daemon output service port"),
 
+    LLAP_TMP_SUBMITWORK_USING_TEZ_AM("hive.llap.tmp.submit.work.using.tez.am", true,""),
+    LLAP_TMP_EXT_CLIENT_NUM_SERVER_HANDLERS("hive.llap.tmp.ext.client.num.server.handlers", 1, ""),
+
     SPARK_CLIENT_FUTURE_TIMEOUT("hive.spark.client.future.timeout",
       "60s", new TimeValidator(TimeUnit.SECONDS),
       "Timeout for requests from Hive client to remote Spark driver."),

http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java b/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
index 97fe2c5..c38dd82 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
@@ -59,6 +59,11 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
   private String pwd;  // ""
   private String query;
 
+  public final String URL_KEY = "llap.if.hs2.connection";
+  public final String QUERY_KEY = "llap.if.query";
+  public final String USER_KEY = "llap.if.user";
+  public final String PWD_KEY = "llap.if.pwd";
+
   private Connection con;
   private Statement stmt;
 
@@ -133,6 +138,11 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
   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();
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/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..653e7e0 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
@@ -7334,6 +7334,16 @@ 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}
@@ -7452,6 +7462,11 @@ public final class LlapDaemonProtocolProtos {
               bitField0_ |= 0x00000200;
               break;
             }
+            case 88: {
+              bitField0_ |= 0x00000400;
+              usingTezAm_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -7799,6 +7814,22 @@ 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_ = "";
@@ -7810,6 +7841,7 @@ 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() {
@@ -7853,6 +7885,9 @@ public final class LlapDaemonProtocolProtos {
       if (((bitField0_ & 0x00000200) == 0x00000200)) {
         output.writeMessage(10, fragmentRuntimeInfo_);
       }
+      if (((bitField0_ & 0x00000400) == 0x00000400)) {
+        output.writeBool(11, usingTezAm_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -7902,6 +7937,10 @@ 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;
@@ -7975,6 +8014,11 @@ 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;
@@ -8028,6 +8072,10 @@ 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;
@@ -8167,6 +8215,8 @@ public final class LlapDaemonProtocolProtos {
           fragmentRuntimeInfoBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000200);
+        usingTezAm_ = true;
+        bitField0_ = (bitField0_ & ~0x00000400);
         return this;
       }
 
@@ -8243,6 +8293,10 @@ 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;
@@ -8299,6 +8353,9 @@ public final class LlapDaemonProtocolProtos {
         if (other.hasFragmentRuntimeInfo()) {
           mergeFragmentRuntimeInfo(other.getFragmentRuntimeInfo());
         }
+        if (other.hasUsingTezAm()) {
+          setUsingTezAm(other.getUsingTezAm());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -9032,6 +9089,39 @@ 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)
     }
 
@@ -14392,7 +14482,7 @@ public final class LlapDaemonProtocolProtos {
       "\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\"\266\002\n\026SubmitW" +
+      " \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_" +
@@ -14400,38 +14490,39 @@ public final class LlapDaemonProtocolProtos {
       "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\"J\n\027SubmitWorkRespo" +
-      "nseProto\022/\n\020submission_state\030\001 \001(\0162\025.Sub" +
-      "missionStateProto\"\205\001\n\036SourceStateUpdated" +
-      "RequestProto\022/\n\020query_identifier\030\001 \001(\0132\025" +
-      ".QueryIdentifierProto\022\020\n\010src_name\030\002 \001(\t\022",
-      " \n\005state\030\003 \001(\0162\021.SourceStateProto\"!\n\037Sou" +
-      "rceStateUpdatedResponseProto\"w\n\031QueryCom" +
-      "pleteRequestProto\022\020\n\010query_id\030\001 \001(\t\022/\n\020q" +
-      "uery_identifier\030\002 \001(\0132\025.QueryIdentifierP" +
-      "roto\022\027\n\014delete_delay\030\004 \001(\003:\0010\"\034\n\032QueryCo" +
-      "mpleteResponseProto\"t\n\035TerminateFragment" +
-      "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"
+      "\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"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -14485,7 +14576,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", });
+              new java.lang.String[] { "ContainerIdString", "AmHost", "AmPort", "TokenIdentifier", "CredentialsBinary", "User", "ApplicationIdString", "AppAttemptNumber", "FragmentSpec", "FragmentRuntimeInfo", "UsingTezAm", });
           internal_static_SubmitWorkResponseProto_descriptor =
             getDescriptor().getMessageTypes().get(8);
           internal_static_SubmitWorkResponseProto_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/llap-common/src/protobuf/LlapDaemonProtocol.proto
----------------------------------------------------------------------
diff --git a/llap-common/src/protobuf/LlapDaemonProtocol.proto b/llap-common/src/protobuf/LlapDaemonProtocol.proto
index 944c96c..e964c5f 100644
--- a/llap-common/src/protobuf/LlapDaemonProtocol.proto
+++ b/llap-common/src/protobuf/LlapDaemonProtocol.proto
@@ -91,6 +91,7 @@ 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 {
@@ -136,11 +137,18 @@ 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/f272acea/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 8621826..40a89cb 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
@@ -14,6 +14,7 @@
 
 package org.apache.hadoop.hive.llap.daemon.impl;
 
+import org.apache.hadoop.hive.llap.LlapOutputFormatService;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryPoolMXBean;
@@ -279,6 +280,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     LOG.info("Setting shuffle port to: " + ShuffleHandler.get().getPort());
     this.shufflePort.set(ShuffleHandler.get().getPort());
     super.serviceStart();
+    LlapOutputFormatService.get();
     LOG.info("LlapDaemon serviceStart complete");
   }
 
@@ -286,6 +288,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     super.serviceStop();
     ShuffleHandler.shutdown();
     shutdown();
+    LlapOutputFormatService.get().stop();
     LOG.info("LlapDaemon shutdown complete");
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/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 d88d82a..d9d216d 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
@@ -103,6 +103,7 @@ 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);
@@ -131,11 +132,11 @@ 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.
-    if (jobToken != null) {
     this.amReporter.registerTask(request.getAmHost(), request.getAmPort(),
         request.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
-    }
     this.metrics = metrics;
     this.requestId = request.getFragmentSpec().getFragmentIdentifierString();
     // TODO Change this to the queryId/Name when that's available.

http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java b/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
new file mode 100644
index 0000000..ecc032d
--- /dev/null
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
@@ -0,0 +1,197 @@
+package org.apache.hadoop.hive.llap.ext;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import com.google.common.base.Preconditions;
+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.SubmitWorkRequestProto;
+import org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol;
+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.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.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, List<TezEvent>> pendingEvents = new ConcurrentHashMap<>();
+
+
+  // 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) {
+    super(LlapTaskUmbilicalExternalClient.class.getName());
+    this.conf = conf;
+    this.umbilical = new LlapTaskUmbilicalExternalImpl();
+    this.tokenIdentifier = tokenIdentifier;
+    this.sessionToken = sessionToken;
+    // 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);
+  }
+
+  @Override
+  public void serviceStart() throws IOException {
+    int numHandlers = HiveConf.getIntVar(conf,
+        HiveConf.ConfVars.LLAP_TMP_EXT_CLIENT_NUM_SERVER_HANDLERS);
+    llapTaskUmbilicalServer = new LlapTaskUmbilicalServer(conf, umbilical, numHandlers, tokenIdentifier, sessionToken);
+  }
+
+  @Override
+  public void serviceStop() {
+    llapTaskUmbilicalServer.shutdownServer();
+    if (this.communicator != null) {
+      this.communicator.stop();
+    }
+  }
+
+
+  /**
+   * Submit the work for actual execution. This should always have the usingTezAm flag disabled
+   * @param submitWorkRequestProto
+   */
+  public void submitWork(final SubmitWorkRequestProto submitWorkRequestProto, String llapHost, int llapPort) {
+    Preconditions.checkArgument(submitWorkRequestProto.getUsingTezAm() == false);
+
+    // Store the actual event first. To be returned on the first heartbeat.
+    Event mrInputEvent = null;
+    // Construct a TezEvent out of this, to send it out on the next heaertbeat
+
+//    submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString()
+
+
+    // 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)) {
+                LOG.info("Fragment: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString() + " rejected. Server Busy.");
+                return;
+              }
+            }
+            LOG.info("DBG: Submitted " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString());
+          }
+
+          @Override
+          public void indicateError(Throwable t) {
+            LOG.error("Failed to submit: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), t);
+          }
+        });
+
+
+
+
+//    // 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);
+
+
+  }
+
+
+
+
+
+
+
+  // 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)
+
+      // Incoming events can be ignored until the point when shuffle needs to be handled, instead of just scans.
+
+
+      TezHeartbeatResponse response = new TezHeartbeatResponse();
+      // Assuming TaskAttemptId and FragmentIdentifierString are the same. Verify this.
+      TezTaskAttemptID taskAttemptId = request.getCurrentTaskAttemptID();
+      LOG.info("ZZZ: DBG: Received heartbeat from taskAttemptId: " + taskAttemptId.toString());
+
+      List<TezEvent> tezEvents = pendingEvents.remove(taskAttemptId.toString());
+
+      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);
+
+      // TODO KKK: Should ideally handle things like Task success notifications.
+      // Can this somehow be hooked into the LlapTaskCommunicator to make testing easy
+
+      return response;
+    }
+
+    @Override
+    public void nodeHeartbeat(Text hostname, int port) throws IOException {
+      // TODO Eventually implement - to handle keep-alive messages from pending work.
+    }
+
+    @Override
+    public void taskKilled(TezTaskAttemptID taskAttemptId) throws IOException {
+      // TODO Eventually implement - to handle preemptions within LLAP daemons.
+    }
+
+    @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/f272acea/llap-server/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
new file mode 100644
index 0000000..dbd591a
--- /dev/null
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
@@ -0,0 +1,57 @@
+package org.apache.hadoop.hive.llap.tezplugins.helpers;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.Token;
+import org.apache.tez.common.security.JobTokenIdentifier;
+import org.apache.tez.common.security.JobTokenSecretManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LlapTaskUmbilicalServer {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LlapTaskUmbilicalServer.class);
+
+  protected volatile Server server;
+  private final InetSocketAddress address;
+  private final AtomicBoolean started = new AtomicBoolean(true);
+
+  public LlapTaskUmbilicalServer(Configuration conf, LlapTaskUmbilicalProtocol umbilical, int numHandlers, String tokenIdentifier, Token<JobTokenIdentifier> token) throws
+      IOException {
+    JobTokenSecretManager jobTokenSecretManager =
+        new JobTokenSecretManager();
+    jobTokenSecretManager.addTokenForJob(tokenIdentifier, token);
+
+    server = new RPC.Builder(conf)
+        .setProtocol(LlapTaskUmbilicalProtocol.class)
+        .setBindAddress("0.0.0.0")
+        .setPort(0)
+        .setInstance(umbilical)
+        .setNumHandlers(numHandlers)
+        .setSecretManager(jobTokenSecretManager).build();
+
+    server.start();
+    this.address = NetUtils.getConnectAddress(server);
+    LOG.info(
+        "Started TaskUmbilicalServer: " + umbilical.getClass().getName() + " at address: " + address +
+            " with numHandlers=" + numHandlers);
+  }
+
+  public InetSocketAddress getAddress() {
+    return this.address;
+  }
+
+  public void shutdownServer() {
+    if (started.get()) { // Primarily to avoid multiple shutdowns.
+      started.set(false);
+      server.stop();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/ql/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
index 4db4d32..d308ec8 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
@@ -18,10 +18,17 @@ package org.apache.hadoop.hive.llap;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Random;
 import java.util.Set;
 
 import javax.security.auth.login.LoginException;
 
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.tez.TezProcessor;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TaskSpecBuilder;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,6 +45,8 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.net.URISyntaxException;
 import java.io.FileNotFoundException;
+import java.util.UUID;
+
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.io.FilenameUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -82,15 +91,14 @@ import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
 import org.apache.tez.runtime.api.events.InputDataInformationEvent;
 import org.apache.tez.runtime.api.events.InputInitializerEvent;
 
-
 import com.google.common.base.Preconditions;
 
 public class LlapInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
 
   private static final Logger LOG = LoggerFactory.getLogger(LlapInputFormat.class);
 
-  private TezWork work;
-  private Schema schema;
+  private final TezWork work;
+  private final Schema schema;
 
   public LlapInputFormat(TezWork tezWork, Schema schema) {
     this.work = tezWork;
@@ -98,22 +106,36 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
   }
 
   // need empty constructor for bean instantiation
-  public LlapInputFormat() {}
+  public LlapInputFormat() {
+    // None of these fields should be required during getRecordReader,
+    // and should not be read.
+    work = null;
+    schema = null;
+  }
 
   /*
    * This proxy record reader has the duty of establishing a connected socket with LLAP, then fire
    * off the work in the split to LLAP and finally return the connected socket back in an
    * LlapRecordReader. The LlapRecordReader class reads the results from the socket.
    */
+  @Override
   public RecordReader<NullWritable, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
 
+    // Calls a static method to ensure none of the object fields are read.
+    return _getRecordReader(split, job, reporter);
+  }
+
+  private static RecordReader _getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws
+      IOException {
     LlapInputSplit llapSplit = (LlapInputSplit)split;
 
     // TODO: push event into LLAP
 
     // this is just the portion that sets up the io to receive data
     String host = split.getLocations()[0];
-    String id = job.get(LlapOutputFormat.LLAP_OF_ID_KEY);
+
+    // TODO: need to construct id here. Format is queryId + "_" + taskIndex
+    String id = "foobar";
 
     HiveConf conf = new HiveConf();
     Socket socket = new Socket(host,
@@ -130,120 +152,8 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     return new LlapRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class);
   }
 
-  /*
-   * getSplits() gets called as part of the GenericUDFGetSplits call to get splits. Here we create
-   * an array of input splits from the work item we have, figure out the location for llap and pass
-   * that back for the submission. getRecordReader method above uses that split info to assign the
-   * work to llap.
-   */
   @Override
   public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
-    // TODO: need to build proto of plan
-
-    DAG dag = DAG.create(work.getName());
-    dag.setCredentials(job.getCredentials());
-    // TODO: set access control? TezTask.setAccessControlsForCurrentUser(dag);
-
-    DagUtils utils = DagUtils.getInstance();
-    Context ctx = new Context(job);
-    MapWork mapWork = (MapWork) work.getAllWork().get(0);
-    // bunch of things get setup in the context based on conf but we need only the MR tmp directory
-    // for the following method.
-    JobConf wxConf = utils.initializeVertexConf(job, ctx, mapWork);
-    Path scratchDir = utils.createTezDir(ctx.getMRScratchDir(), job);
-    FileSystem fs = scratchDir.getFileSystem(job);
-    try {
-      LocalResource appJarLr = createJarLocalResource(utils.getExecJarPathLocal(), utils, job);
-      Vertex wx = utils.createVertex(wxConf, mapWork, scratchDir, appJarLr,
-          new ArrayList<LocalResource>(), fs, ctx, false, work,
-          work.getVertexType(mapWork));
-      dag.addVertex(wx);
-      utils.addCredentials(mapWork, dag);
-
-      // we have the dag now proceed to get the splits:
-      HiveSplitGenerator splitGenerator = new HiveSplitGenerator(null);
-      splitGenerator.initializeSplitGenerator(wxConf, mapWork);
-      List<Event> eventList = splitGenerator.initialize();
-
-      // hack - just serializing with kryo for now. This needs to be done properly
-      InputSplit[] result = new InputSplit[eventList.size()];
-      int i = 0;
-      ByteArrayOutputStream bos = new ByteArrayOutputStream(10240);
-
-      InputConfigureVertexTasksEvent configureEvent = (InputConfigureVertexTasksEvent)
-	eventList.remove(0);
-
-      List<TaskLocationHint> hints = configureEvent.getLocationHint().getTaskLocationHints();
-      for (Event event: eventList) {
-	TaskLocationHint hint = hints.remove(0);
-        Set<String> hosts = hint.getHosts();
-	SplitLocationInfo[] locations = new SplitLocationInfo[hosts.size()];
-
-	int j = 0;
-	for (String host: hosts) {
-	  locations[j++] = new SplitLocationInfo(host,false);
-	}
-
-	bos.reset();
-	Kryo kryo = SerializationUtilities.borrowKryo();
-	SerializationUtilities.serializeObjectByKryo(kryo, event, bos);
-	SerializationUtilities.releaseKryo(kryo);
-	result[i++] = new LlapInputSplit(bos.toByteArray(), locations, schema);
-      }
-      return result;
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-  }
-
-  /**
-   * Returns a local resource representing a jar. This resource will be used to execute the plan on
-   * the cluster.
-   *
-   * @param localJarPath
-   *          Local path to the jar to be localized.
-   * @return LocalResource corresponding to the localized hive exec resource.
-   * @throws IOException
-   *           when any file system related call fails.
-   * @throws LoginException
-   *           when we are unable to determine the user.
-   * @throws URISyntaxException
-   *           when current jar location cannot be determined.
-   */
-  private LocalResource createJarLocalResource(String localJarPath, DagUtils utils,
-      Configuration conf)
-    throws IOException, LoginException, IllegalArgumentException, FileNotFoundException {
-    FileStatus destDirStatus = utils.getHiveJarDirectory(conf);
-    assert destDirStatus != null;
-    Path destDirPath = destDirStatus.getPath();
-
-    Path localFile = new Path(localJarPath);
-    String sha = getSha(localFile, conf);
-
-    String destFileName = localFile.getName();
-
-    // Now, try to find the file based on SHA and name. Currently we require exact name match.
-    // We could also allow cutting off versions and other stuff provided that SHA matches...
-    destFileName = FilenameUtils.removeExtension(destFileName) + "-" + sha
-      + FilenameUtils.EXTENSION_SEPARATOR + FilenameUtils.getExtension(destFileName);
-
-    // TODO: if this method is ever called on more than one jar, getting the dir and the
-    // list need to be refactored out to be done only once.
-    Path destFile = new Path(destDirPath.toString() + "/" + destFileName);
-    return utils.localizeResource(localFile, destFile, LocalResourceType.FILE, conf);
-  }
-
-  private String getSha(Path localFile, Configuration conf)
-    throws IOException, IllegalArgumentException {
-    InputStream is = null;
-    try {
-      FileSystem localFs = FileSystem.getLocal(conf);
-      is = localFs.open(localFile);
-      return DigestUtils.sha256Hex(is);
-    } finally {
-      if (is != null) {
-        is.close();
-      }
-    }
+    throw new IOException("These are not the splits you are looking for.");
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
index 78dbb34..4249a16 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
@@ -16,49 +16,49 @@
  */
 package org.apache.hadoop.hive.llap;
 
-import java.io.IOException;
 import java.io.DataInput;
 import java.io.DataOutput;
-import java.io.DataInputStream;
-import java.io.ByteArrayInputStream;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputFormat;
-import org.apache.hadoop.mapred.RecordWriter;
-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.hive.llap.io.api.LlapProxy;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.SplitLocationInfo;
-import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
+import java.io.IOException;
+
 import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
+import org.apache.hadoop.mapred.SplitLocationInfo;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.transport.AutoExpandingBufferWriteTransport;
 import org.apache.thrift.transport.AutoExpandingBuffer;
-
-import com.google.common.base.Preconditions;
+import org.apache.thrift.transport.AutoExpandingBufferWriteTransport;
 
 public class LlapInputSplit implements InputSplitWithLocationInfo {
 
-  byte[] queryFragment;
+  byte[] planBytes;
+  byte[] fragmentBytes;
   SplitLocationInfo[] locations;
   Schema schema;
 
-  public LlapInputSplit() {}
 
-  public LlapInputSplit(byte[] queryFragment, SplitLocationInfo[] locations, Schema schema) {
-    this.queryFragment = queryFragment;
+  // // Static
+  // ContainerIdString
+  // DagName
+  // VertexName
+  // FragmentNumber
+  // AttemptNumber - always 0
+  // FragmentIdentifierString - taskAttemptId
+
+  // ProcessorDescsriptor
+  // InputSpec
+  // OutputSpec
+
+  // Tokens
+
+  // // Dynamic
+  //
+
+  public LlapInputSplit() {
+  }
+
+  public LlapInputSplit(byte[] planBytes, byte[] fragmentBytes, SplitLocationInfo[] locations, Schema schema) {
+    this.planBytes = planBytes;
+    this.fragmentBytes = fragmentBytes;
     this.locations = locations;
     this.schema = schema;
   }
@@ -83,8 +83,11 @@ public class LlapInputSplit implements InputSplitWithLocationInfo {
 
   @Override
   public void write(DataOutput out) throws IOException {
-    out.writeInt(queryFragment.length);
-    out.write(queryFragment);
+    out.writeInt(planBytes.length);
+    out.write(planBytes);
+
+    out.writeInt(fragmentBytes.length);
+    out.write(fragmentBytes);
 
     out.writeInt(locations.length);
     for (int i = 0; i < locations.length; ++i) {
@@ -108,11 +111,13 @@ public class LlapInputSplit implements InputSplitWithLocationInfo {
 
   @Override
   public void readFields(DataInput in) throws IOException {
-    byte[] queryFragment;
-
     int length = in.readInt();
-    queryFragment = new byte[length];
-    in.readFully(queryFragment);
+    planBytes = new byte[length];
+    in.readFully(planBytes);
+
+    length = in.readInt();
+    fragmentBytes = new byte[length];
+    in.readFully(fragmentBytes);
 
     length = in.readInt();
     locations = new SplitLocationInfo[length];
@@ -124,7 +129,8 @@ public class LlapInputSplit implements InputSplitWithLocationInfo {
     length = in.readInt();
 
     try {
-      AutoExpandingBufferWriteTransport transport = new AutoExpandingBufferWriteTransport(length, 2d);
+      AutoExpandingBufferWriteTransport transport =
+          new AutoExpandingBufferWriteTransport(length, 2d);
       AutoExpandingBuffer buf = transport.getBuf();
       in.readFully(buf.array(), 0, length);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/ql/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java b/ql/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
new file mode 100644
index 0000000..a9a3738
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
@@ -0,0 +1,65 @@
+package org.apache.hadoop.hive.llap;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+
+public class SubmitWorkInfo implements Writable {
+
+  private TaskSpec taskSpec;
+  private ApplicationId fakeAppId;
+
+  public SubmitWorkInfo(TaskSpec taskSpec, ApplicationId fakeAppId) {
+    this.taskSpec = taskSpec;
+    this.fakeAppId = fakeAppId;
+  }
+
+  // Empty constructor for writable etc.
+  public SubmitWorkInfo() {
+  }
+
+  public TaskSpec getTaskSpec() {
+    return taskSpec;
+  }
+
+  public ApplicationId getFakeAppId() {
+    return fakeAppId;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    taskSpec.write(out);
+    out.writeLong(fakeAppId.getClusterTimestamp());
+    out.writeInt(fakeAppId.getId());
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    taskSpec = new TaskSpec();
+    taskSpec.readFields(in);
+    long appIdTs = in.readLong();
+    int appIdId = in.readInt();
+    fakeAppId = ApplicationId.newInstance(appIdTs, appIdId);
+  }
+
+  public static byte[] toBytes(SubmitWorkInfo submitWorkInfo) throws IOException {
+    DataOutputBuffer dob = new DataOutputBuffer();
+    submitWorkInfo.write(dob);
+    return dob.getData();
+  }
+
+  public SubmitWorkInfo fromBytes(byte[] submitWorkInfoBytes) throws IOException {
+    DataInputBuffer dib = new DataInputBuffer();
+    dib.reset(submitWorkInfoBytes, 0, submitWorkInfoBytes.length);
+    SubmitWorkInfo submitWorkInfo = new SubmitWorkInfo();
+    submitWorkInfo.readFields(dib);
+    return submitWorkInfo;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
index b0cda82..011e459 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
@@ -33,6 +33,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.llap.registry.ServiceInstance;
+import org.apache.hadoop.hive.llap.registry.ServiceInstanceSet;
+import org.apache.hadoop.hive.llap.registry.impl.LlapRegistryService;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.plan.MapWork;
 import org.apache.hadoop.hive.serde2.SerDeException;
@@ -44,6 +47,7 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.split.SplitLocationProvider;
 import org.apache.hadoop.mapreduce.split.TezMapReduceSplitsGrouper;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.api.VertexLocationHint;
@@ -82,10 +86,30 @@ public class HiveSplitGenerator extends InputInitializer {
   private final SplitGrouper splitGrouper = new SplitGrouper();
   private SplitLocationProvider splitLocationProvider = null;
 
-  public void initializeSplitGenerator(Configuration conf, MapWork work) {
+  
+  // TODO RSHACK This entire method needs to be reworked. Put back final fields, separate into reusable components etc.
+  public void initializeSplitGenerator(Configuration conf, MapWork work) throws IOException {
+
     this.conf = conf;
     this.work = work;
-    this.jobConf = new JobConf(conf);
+
+    // TODO RSHACK - assuming grouping enabled always.
+    userPayloadProto = MRInputUserPayloadProto.newBuilder().setGroupingEnabled(true).build();
+
+    this.splitLocationProvider = Utils.getSplitLocationProvider(conf, LOG);
+    LOG.info("SplitLocationProvider: " + splitLocationProvider);
+
+    // Read all credentials into the credentials instance stored in JobConf.
+    ShimLoader.getHadoopShims().getMergedCredentials(jobConf);
+
+    this.work = Utilities.getMapWork(jobConf);
+
+    // Events can start coming in the moment the InputInitializer is created. The pruner
+    // must be setup and initialized here so that it sets up it's structures to start accepting events.
+    // Setting it up in initialize leads to a window where events may come in before the pruner is
+    // initialized, which may cause it to drop events.
+    // TODO RSHACK - No dynamic partition pruning
+    pruner = null;
   }
 
   public HiveSplitGenerator(InputInitializerContext initializerContext) throws IOException,
@@ -129,7 +153,9 @@ public class HiveSplitGenerator extends InputInitializer {
           conf.getBoolean("mapreduce.tez.input.initializer.serialize.event.payload", true);
 
       // perform dynamic partition pruning
-      pruner.prune();
+      if (pruner != null) {
+        pruner.prune();
+      }
 
       InputSplitInfoMem inputSplitInfo = null;
       boolean generateConsistentSplits = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_TEZ_GENERATE_CONSISTENT_SPLITS);
@@ -142,9 +168,20 @@ public class HiveSplitGenerator extends InputInitializer {
           (InputFormat<?, ?>) ReflectionUtils.newInstance(JavaUtils.loadClass(realInputFormatName),
               jobConf);
 
-        int totalResource = getContext().getTotalAvailableResource().getMemory();
-        int taskResource = getContext().getVertexTaskResource().getMemory();
-        int availableSlots = totalResource / taskResource;
+        int totalResource = 0;
+        int taskResource = 0;
+        int availableSlots = 0;
+        // FIXME. Do the right thing Luke.
+        if (getContext() == null) {
+          // for now, totalResource = taskResource for llap
+          availableSlots = 1;
+        }
+
+        if (getContext() != null) {
+          totalResource = getContext().getTotalAvailableResource().getMemory();
+          taskResource = getContext().getVertexTaskResource().getMemory();
+          availableSlots = totalResource / taskResource;
+        }
 
         if (HiveConf.getLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, 1) <= 1) {
           // broken configuration from mapred-default.xml

http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
index 0584ad8..3fe70ab 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.llap.LlapOutputFormat;
 import org.apache.hadoop.hive.ql.exec.DummyStoreOperator;
 import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator;
 import org.apache.hadoop.hive.ql.exec.MapOperator;
@@ -94,6 +95,7 @@ public class MapRecordProcessor extends RecordProcessor {
     super(jconf, context);
     String queryId = HiveConf.getVar(jconf, HiveConf.ConfVars.HIVEQUERYID);
     if (LlapProxy.isDaemon()) { // do not cache plan
+      jconf.set(LlapOutputFormat.LLAP_OF_ID_KEY, queryId + "_" + context.getTaskIndex());
       cache = new org.apache.hadoop.hive.ql.exec.mr.ObjectCache();
     } else {
       cache = ObjectCacheFactory.getCache(jconf, queryId);

http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
index 598520c..0997233 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
@@ -1315,12 +1315,15 @@ public class TypeCheckProcFactory {
       try {
         return getXpathOrFuncExprNodeDesc(expr, isFunction, children, ctx);
       } catch (UDFArgumentTypeException e) {
+        LOG.error("UDFArgumentTypeException: ", e);
         throw new SemanticException(ErrorMsg.INVALID_ARGUMENT_TYPE.getMsg(expr
             .getChild(childrenBegin + e.getArgumentId()), e.getMessage()));
       } catch (UDFArgumentLengthException e) {
+        LOG.error("UDFArgumentLengthException: ", e);
         throw new SemanticException(ErrorMsg.INVALID_ARGUMENT_LENGTH.getMsg(
             expr, e.getMessage()));
       } catch (UDFArgumentException e) {
+        LOG.error("UDFArgumentException: ", e);
         throw new SemanticException(ErrorMsg.INVALID_ARGUMENT.getMsg(expr, e
             .getMessage()));
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
index 3b7dcd9..9c7e1f2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
@@ -18,6 +18,20 @@
 
 package org.apache.hadoop.hive.ql.udf.generic;
 
+import org.apache.hadoop.hive.llap.LlapInputSplit;
+import org.apache.hadoop.hive.llap.SubmitWorkInfo;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
+
+import javax.security.auth.login.LoginException;
+
 import java.util.Arrays;
 import java.util.List;
 import java.util.ArrayList;
@@ -28,6 +42,17 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.DataOutput;
 
+import com.esotericsoftware.kryo.Kryo;
+import java.io.ByteArrayOutputStream;
+import java.io.OutputStream;
+import java.io.InputStream;
+
+import org.apache.hadoop.hive.ql.exec.tez.TezProcessor;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TaskSpecBuilder;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -69,6 +94,55 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.metastore.api.Schema;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URISyntaxException;
+import java.io.FileNotFoundException;
+import java.util.UUID;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.SplitLocationInfo;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.runtime.api.Event;
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.exec.tez.DagUtils;
+import org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator;
+import org.apache.hadoop.hive.ql.plan.MapWork;
+import org.apache.hadoop.hive.ql.plan.TezWork;
+import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.llap.configuration.LlapConfiguration;
+import org.apache.tez.dag.api.TaskLocationHint;
+import org.apache.tez.dag.api.VertexLocationHint;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.mapreduce.hadoop.InputSplitInfoMem;
+import org.apache.tez.mapreduce.hadoop.MRInputHelpers;
+import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRInputUserPayloadProto;
+import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitProto;
+import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitsProto;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.InputInitializer;
+import org.apache.tez.runtime.api.InputInitializerContext;
+import org.apache.tez.runtime.api.InputSpecUpdate;
+import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
+import org.apache.tez.runtime.api.events.InputInitializerEvent;
 
 /**
  * GenericUDFGetSplits.
@@ -177,7 +251,6 @@ public class GenericUDFGetSplits extends GenericUDF {
     }
 
     Path data = null;
-    InputFormat inp = null;
     String ifc = null;
 
     TezWork tezWork = ((TezTask)roots.get(0)).getWork();
@@ -214,33 +287,13 @@ public class GenericUDFGetSplits extends GenericUDF {
       }
 
       tezWork = ((TezTask)roots.get(0)).getWork();
-
-      // Table table = db.getTable(tableName);
-      // if (table.isPartitioned()) {
-      //   throw new UDFArgumentException("Table " + tableName + " is partitioned.");
-      // }
-      // data = table.getDataLocation();
-      // LOG.info("looking at: "+data);
-
-      // ifc = table.getInputFormatClass().toString();
-
-      // inp = ReflectionUtils.newInstance(table.getInputFormatClass(), jc);
     }
 
     MapWork w = (MapWork)tezWork.getAllWork().get(0);
-    inp = new LlapInputFormat(tezWork, schema);
     ifc = LlapInputFormat.class.toString();
 
     try {
-      if (inp instanceof JobConfigurable) {
-        ((JobConfigurable) inp).configure(jc);
-      }
-
-      if (inp instanceof FileInputFormat) {
-        ((FileInputFormat) inp).addInputPath(jc, data);
-      }
-
-      for (InputSplit s: inp.getSplits(jc, num)) {
+      for (InputSplit s: getSplits(jc, num, tezWork, schema)) {
         Object[] os = new Object[3];
         os[0] = ifc;
         os[1] = s.getClass().toString();
@@ -257,6 +310,133 @@ public class GenericUDFGetSplits extends GenericUDF {
     return retArray;
   }
 
+  public InputSplit[] getSplits(JobConf job, int numSplits, TezWork work, Schema schema) throws IOException {
+    DAG dag = DAG.create(work.getName());
+    dag.setCredentials(job.getCredentials());
+    // TODO: set access control? TezTask.setAccessControlsForCurrentUser(dag);
+
+    DagUtils utils = DagUtils.getInstance();
+    Context ctx = new Context(job);
+    MapWork mapWork = (MapWork) work.getAllWork().get(0);
+    // bunch of things get setup in the context based on conf but we need only the MR tmp directory
+    // for the following method.
+    JobConf wxConf = utils.initializeVertexConf(job, ctx, mapWork);
+    Path scratchDir = utils.createTezDir(ctx.getMRScratchDir(), job);
+    FileSystem fs = scratchDir.getFileSystem(job);
+    try {
+      LocalResource appJarLr = createJarLocalResource(utils.getExecJarPathLocal(), utils, job);
+      Vertex wx = utils.createVertex(wxConf, mapWork, scratchDir, appJarLr,
+          new ArrayList<LocalResource>(), fs, ctx, false, work,
+          work.getVertexType(mapWork));
+      String vertexName = wx.getName();
+      dag.addVertex(wx);
+      utils.addCredentials(mapWork, dag);
+
+      // we have the dag now proceed to get the splits:
+      HiveSplitGenerator splitGenerator = new HiveSplitGenerator(null);
+      Preconditions.checkState(HiveConf.getBoolVar(wxConf,
+          HiveConf.ConfVars.HIVE_TEZ_GENERATE_CONSISTENT_SPLITS));
+      Preconditions.checkState(HiveConf.getBoolVar(wxConf,
+          HiveConf.ConfVars.LLAP_CLIENT_CONSISTENT_SPLITS));
+      splitGenerator.initializeSplitGenerator(wxConf, mapWork);
+      List<Event> eventList = splitGenerator.initialize();
+
+      // hack - just serializing with kryo for now. This needs to be done properly
+      InputSplit[] result = new InputSplit[eventList.size()];
+      ByteArrayOutputStream bos = new ByteArrayOutputStream(10240);
+
+      InputConfigureVertexTasksEvent configureEvent = (InputConfigureVertexTasksEvent) eventList.get(0);
+
+      List<TaskLocationHint> hints = configureEvent.getLocationHint().getTaskLocationHints();
+
+      Preconditions.checkState(hints.size() == eventList.size() -1);
+
+      LOG.info("DBG: Number of splits: " + (eventList.size() - 1));
+      for (int i = 1 ; i < eventList.size() ; i++) {
+        // Creating the TezEvent here itself, since it's easy to serialize.
+        Event event = eventList.get(i);
+        TaskLocationHint hint = hints.get(i-1);
+        Set<String> hosts = hint.getHosts();
+        LOG.info("DBG: Using locations: " + hosts.toString());
+        if (hosts.size() != 1) {
+          LOG.warn("DBG: Bad # of locations: " + hosts.size());
+        }
+        SplitLocationInfo[] locations = new SplitLocationInfo[hosts.size()];
+
+        int j = 0;
+        for (String host : hosts) {
+          locations[j++] = new SplitLocationInfo(host, false);
+        }
+
+        bos.reset();
+        Kryo kryo = SerializationUtilities.borrowKryo();
+        SerializationUtilities.serializeObjectByKryo(kryo, event, bos);
+        SerializationUtilities.releaseKryo(kryo);
+
+        TaskSpec taskSpec = new TaskSpecBuilder().constructTaskSpec(dag, vertexName, eventList.size() - 1);
+        ApplicationId fakeApplicationId = ApplicationId.newInstance(new Random().nextInt(), 0);
+        SubmitWorkInfo submitWorkInfo = new SubmitWorkInfo(taskSpec, fakeApplicationId);
+        byte[] submitWorkBytes = SubmitWorkInfo.toBytes(submitWorkInfo);
+
+        result[i-1] = new LlapInputSplit(submitWorkBytes, bos.toByteArray(), locations, schema);
+      }
+      return result;
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+    /**
+   * Returns a local resource representing a jar. This resource will be used to execute the plan on
+   * the cluster.
+   *
+   * @param localJarPath
+   *          Local path to the jar to be localized.
+   * @return LocalResource corresponding to the localized hive exec resource.
+   * @throws IOException
+   *           when any file system related call fails.
+   * @throws LoginException
+   *           when we are unable to determine the user.
+   * @throws URISyntaxException
+   *           when current jar location cannot be determined.
+   */
+  private LocalResource createJarLocalResource(String localJarPath, DagUtils utils,
+      Configuration conf)
+    throws IOException, LoginException, IllegalArgumentException, FileNotFoundException {
+    FileStatus destDirStatus = utils.getHiveJarDirectory(conf);
+    assert destDirStatus != null;
+    Path destDirPath = destDirStatus.getPath();
+
+    Path localFile = new Path(localJarPath);
+    String sha = getSha(localFile, conf);
+
+    String destFileName = localFile.getName();
+
+    // Now, try to find the file based on SHA and name. Currently we require exact name match.
+    // We could also allow cutting off versions and other stuff provided that SHA matches...
+    destFileName = FilenameUtils.removeExtension(destFileName) + "-" + sha
+      + FilenameUtils.EXTENSION_SEPARATOR + FilenameUtils.getExtension(destFileName);
+
+    // TODO: if this method is ever called on more than one jar, getting the dir and the
+    // list need to be refactored out to be done only once.
+    Path destFile = new Path(destDirPath.toString() + "/" + destFileName);
+    return utils.localizeResource(localFile, destFile, LocalResourceType.FILE, conf);
+  }
+
+  private String getSha(Path localFile, Configuration conf)
+    throws IOException, IllegalArgumentException {
+    InputStream is = null;
+    try {
+      FileSystem localFs = FileSystem.getLocal(conf);
+      is = localFs.open(localFile);
+      return DigestUtils.sha256Hex(is);
+    } finally {
+      if (is != null) {
+        is.close();
+      }
+    }
+  }
+
   @Override
   public String getDisplayString(String[] children) {
     assert children.length == 2;

http://git-wip-us.apache.org/repos/asf/hive/blob/f272acea/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java b/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
new file mode 100644
index 0000000..d0c7c5a
--- /dev/null
+++ b/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
@@ -0,0 +1,45 @@
+package org.apache.tez.dag.api;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+
+// Proxy class within the tez.api package to access package private methods.
+public class TaskSpecBuilder {
+
+  public TaskSpec constructTaskSpec(DAG dag, String vertexName, int numSplits) {
+    Vertex vertex = dag.getVertex(vertexName);
+    ProcessorDescriptor processorDescriptor = vertex.getProcessorDescriptor();
+    List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> inputs =
+        vertex.getInputs();
+    List<RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>> outputs =
+        vertex.getOutputs();
+
+    // TODO RSHACK - for now these must be of size 1.
+    Preconditions.checkState(inputs.size() == 1);
+    Preconditions.checkState(outputs.size() == 1);
+
+    List<InputSpec> inputSpecs = new ArrayList<>();
+    for (RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input : inputs) {
+      InputSpec inputSpec = new InputSpec(input.getName(), input.getIODescriptor(), 1);
+      inputSpecs.add(inputSpec);
+    }
+
+    List<OutputSpec> outputSpecs = new ArrayList<>();
+    for (RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor> output : outputs) {
+      OutputSpec outputSpec = new OutputSpec(output.getName(), output.getIODescriptor(), 1);
+      outputSpecs.add(outputSpec);
+    }
+
+    TaskSpec taskSpec = TaskSpec
+        .createBaseTaskSpec(dag.getName(), vertexName, numSplits, processorDescriptor, inputSpecs,
+            outputSpecs, null);
+
+    return taskSpec;
+  }
+
+}


[34/39] 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/master
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
----------------------------------------------------------------------


[17/39] hive git commit: HIVE-13519: Allow LlapRecordReader to parse/output rows

Posted by jd...@apache.org.
HIVE-13519: Allow LlapRecordReader to parse/output rows


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/fc7343dd
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/fc7343dd
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/fc7343dd

Branch: refs/heads/master
Commit: fc7343dd12ac152267615e6ac67238ee06326452
Parents: 8f6b28a
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu Apr 14 14:30:45 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu Apr 14 14:30:45 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/jdbc/TestLlapInputSplit.java    |  21 +-
 .../apache/hive/jdbc/TestJdbcWithMiniLlap.java  |  38 ++--
 jdbc/pom.xml                                    |   5 +
 .../apache/hive/jdbc/LlapBaseInputFormat.java   | 135 ++++++++++++
 .../src/java/org/apache/hive/jdbc/LlapDump.java |  11 +-
 .../org/apache/hive/jdbc/LlapInputFormat.java   | 135 ------------
 .../apache/hive/jdbc/LlapRowInputFormat.java    |  34 +++
 llap-common/pom.xml                             |  21 ++
 .../org/apache/hadoop/hive/llap/FieldDesc.java  |  63 ++++++
 .../hadoop/hive/llap/LlapRowRecordReader.java   | 155 ++++++++++++++
 .../java/org/apache/hadoop/hive/llap/Row.java   | 166 +++++++++++++++
 .../org/apache/hadoop/hive/llap/Schema.java     |  76 +++++++
 .../org/apache/hadoop/hive/llap/TypeDesc.java   | 108 ++++++++++
 .../org/apache/hadoop/hive/llap/TestRow.java    |  92 +++++++++
 .../hadoop/hive/llap/LlapInputFormat.java       |  27 ++-
 .../hadoop/hive/llap/LlapBaseRecordReader.java  | 205 ++++++++++++++++++
 .../apache/hadoop/hive/llap/LlapInputSplit.java |  27 +--
 .../hadoop/hive/llap/LlapRecordReader.java      | 206 -------------------
 .../ql/udf/generic/GenericUDTFGetSplits.java    |  87 +++++++-
 .../hadoop/hive/llap/TestLlapOutputFormat.java  |   6 +-
 20 files changed, 1205 insertions(+), 413 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
index 338930e..366e326 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
@@ -10,8 +10,10 @@ import java.util.HashMap;
 
 import org.apache.hadoop.io.Text;
 
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.llap.Schema;
+import org.apache.hadoop.hive.llap.FieldDesc;
+import org.apache.hadoop.hive.llap.TypeDesc;
+
 import org.apache.hadoop.mapred.SplitLocationInfo;
 import org.junit.After;
 import org.junit.Before;
@@ -32,14 +34,11 @@ public class TestLlapInputSplit {
         new SplitLocationInfo("location1", false),
         new SplitLocationInfo("location2", false),
     };
-    ArrayList<FieldSchema> fields = new ArrayList<FieldSchema>();
-    fields.add(new FieldSchema("col1", "string", "comment1"));
-    fields.add(new FieldSchema("col2", "int", "comment2"));
-    HashMap<String, String> properties = new HashMap<String, String>();
-    properties.put("key1", "val1");
-    Schema schema = new Schema(
-        fields,
-        properties);
+
+    ArrayList<FieldDesc> colDescs = new ArrayList<FieldDesc>();
+    colDescs.add(new FieldDesc("col1", new TypeDesc(TypeDesc.Type.STRING)));
+    colDescs.add(new FieldDesc("col2", new TypeDesc(TypeDesc.Type.INT)));
+    Schema schema = new Schema(colDescs);
 
     org.apache.hadoop.hive.llap.LlapInputSplit split1 = new org.apache.hadoop.hive.llap.LlapInputSplit(
         splitNum,
@@ -94,7 +93,7 @@ public class TestLlapInputSplit {
       assertEquals(locationInfo1[idx].isOnDisk(), locationInfo2[idx].isOnDisk());
     }
     assertArrayEquals(split1.getLocations(), split2.getLocations());
-    assertEquals(split1.getSchema(), split2.getSchema());
+    assertEquals(split1.getSchema().toString(), split2.getSchema().toString());
     assertEquals(split1.getLlapUser(), split2.getLlapUser());
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
index 98daab4..deeac2e 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
@@ -60,16 +60,16 @@ import org.apache.hadoop.mapred.RecordReader;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.llap.LlapRecordReader;
-import org.apache.hadoop.hive.metastore.ObjectStore;
-import org.apache.hadoop.hive.metastore.api.Schema;
-import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.llap.LlapRowRecordReader;
+import org.apache.hadoop.hive.llap.Row;
+import org.apache.hadoop.hive.llap.Schema;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
 
 import org.apache.hive.jdbc.miniHS2.MiniHS2;
 import org.apache.hive.jdbc.miniHS2.MiniHS2.MiniClusterType;
-import org.apache.hive.jdbc.LlapInputFormat;
+import org.apache.hive.jdbc.LlapBaseInputFormat;
+import org.apache.hive.jdbc.LlapRowInputFormat;
 
 import org.datanucleus.ClassLoaderResolver;
 import org.datanucleus.NucleusContext;
@@ -109,8 +109,6 @@ public class TestJdbcWithMiniLlap {
 
     conf.addResource(new URL("file://" + new File(confDir).toURI().getPath()
         + "/tez-site.xml"));
-    conf.addResource(new URL("file://" + new File(confDir).toURI().getPath()
-        + "/llap-daemon-site.xml"));
 
     miniHS2 = new MiniHS2(conf, MiniClusterType.LLAP);
 
@@ -202,10 +200,14 @@ public class TestJdbcWithMiniLlap {
     String user = System.getProperty("user.name");
     String pwd = user;
 
-    LlapInputFormat inputFormat = new LlapInputFormat(url, user, pwd, query);
+    LlapRowInputFormat inputFormat = new LlapRowInputFormat();
 
     // Get splits
     JobConf job = new JobConf(conf);
+    job.set(LlapBaseInputFormat.URL_KEY, url);
+    job.set(LlapBaseInputFormat.USER_KEY, user);
+    job.set(LlapBaseInputFormat.PWD_KEY, pwd);
+    job.set(LlapBaseInputFormat.QUERY_KEY, query);
 
     InputSplit[] splits = inputFormat.getSplits(job, numSplits);
     assertTrue(splits.length > 0);
@@ -216,10 +218,12 @@ public class TestJdbcWithMiniLlap {
     for (InputSplit split : splits) {
       System.out.println("Processing split " + split.getLocations());
 
-      RecordReader<NullWritable, Text> reader = inputFormat.getRecordReader(split, job, null);
-      if (reader instanceof LlapRecordReader && first) {
-        Schema schema = ((LlapRecordReader)reader).getSchema();
+      int numColumns = 2;
+      RecordReader<NullWritable, Row> reader = inputFormat.getRecordReader(split, job, null);
+      if (reader instanceof LlapRowRecordReader && first) {
+        Schema schema = ((LlapRowRecordReader) reader).getSchema();
         System.out.println(""+schema);
+        assertEquals(numColumns, schema.getColumns().size());
       }
 
       if (first) {
@@ -228,9 +232,15 @@ public class TestJdbcWithMiniLlap {
         first = false;
       }
 
-      Text value = reader.createValue();
-      while (reader.next(NullWritable.get(), value)) {
-        System.out.println(value);
+      Row row = reader.createValue();
+      while (reader.next(NullWritable.get(), row)) {
+        for (int idx = 0; idx < numColumns; idx++) {
+          if (idx > 0) {
+            System.out.print(", ");
+          }
+          System.out.print(row.getValue(idx));
+        }
+        System.out.println("");
         ++rowCount;
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index 2be8c30..c99a351 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -70,6 +70,11 @@
       <artifactId>hive-service-rpc</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-llap-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
     <!-- inter-project -->
     <dependency>
       <groupId>org.apache.httpcomponents</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/jdbc/src/java/org/apache/hive/jdbc/LlapBaseInputFormat.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapBaseInputFormat.java b/jdbc/src/java/org/apache/hive/jdbc/LlapBaseInputFormat.java
new file mode 100644
index 0000000..a0ddeaa
--- /dev/null
+++ b/jdbc/src/java/org/apache/hive/jdbc/LlapBaseInputFormat.java
@@ -0,0 +1,135 @@
+/*
+ * 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.jdbc;
+
+import java.util.ArrayList;
+import java.util.List;
+
+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 org.apache.hadoop.fs.FileSystem;
+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.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.Progressable;
+
+import com.google.common.base.Preconditions;
+
+public class LlapBaseInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
+
+  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;
+    return llapSplit.getInputFormat().getRecordReader(llapSplit.getSplit(), job, reporter);
+  }
+
+  @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(3));
+        InputSplitWithLocationInfo is = (InputSplitWithLocationInfo)Class.forName(res.getString(2)).newInstance();
+        is.readFields(in);
+        ins.add(new LlapInputSplit(is, res.getString(1)));
+      }
+
+      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
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java b/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
index 7ed0a0e..4c3c3ab 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/LlapDump.java
@@ -49,10 +49,9 @@ import org.apache.hadoop.mapred.RecordWriter;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.hive.metastore.api.Schema;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
-import org.apache.hadoop.hive.llap.LlapRecordReader;
-import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.llap.LlapBaseRecordReader;
+import org.apache.hadoop.hive.llap.Schema;
 
 public class LlapDump {
 
@@ -98,7 +97,7 @@ public class LlapDump {
     System.out.println("user: "+user);
     System.out.println("query: "+query);
 
-    LlapInputFormat format = new LlapInputFormat(url, user, pwd, query);
+    LlapBaseInputFormat format = new LlapBaseInputFormat(url, user, pwd, query);
     JobConf job = new JobConf();
 
     InputSplit[] splits = format.getSplits(job, Integer.parseInt(numSplits));
@@ -113,8 +112,8 @@ public class LlapDump {
         LOG.info("Processing input split s from " + Arrays.toString(s.getLocations()));
         RecordReader<NullWritable, Text> reader = format.getRecordReader(s, job, null);
 
-        if (reader instanceof LlapRecordReader && first) {
-          Schema schema = ((LlapRecordReader)reader).getSchema();
+        if (reader instanceof LlapBaseRecordReader && first) {
+          Schema schema = ((LlapBaseRecordReader)reader).getSchema();
           System.out.println(""+schema);
         }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java b/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
deleted file mode 100644
index 9a7c16d..0000000
--- a/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.jdbc;
-
-import java.util.ArrayList;
-import java.util.List;
-
-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 org.apache.hadoop.fs.FileSystem;
-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.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.util.Progressable;
-
-import com.google.common.base.Preconditions;
-
-public class LlapInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
-
-  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 final String URL_KEY = "llap.if.hs2.connection";
-  public final String QUERY_KEY = "llap.if.query";
-  public final String USER_KEY = "llap.if.user";
-  public final String PWD_KEY = "llap.if.pwd";
-
-  public final String SPLIT_QUERY = "select get_splits(\"%s\",%d)";
-
-  private Connection con;
-  private Statement stmt;
-
-  public LlapInputFormat(String url, String user, String pwd, String query) {
-    this.url = url;
-    this.user = user;
-    this.pwd = pwd;
-    this.query = query;
-  }
-
-  public LlapInputFormat() {}
-
-
-  @Override
-  public RecordReader<NullWritable, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
-    LlapInputSplit llapSplit = (LlapInputSplit) split;
-    return llapSplit.getInputFormat().getRecordReader(llapSplit.getSplit(), job, reporter);
-  }
-
-  @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(3));
-        InputSplitWithLocationInfo is = (InputSplitWithLocationInfo)Class.forName(res.getString(2)).newInstance();
-        is.readFields(in);
-        ins.add(new LlapInputSplit(is, res.getString(1)));
-      }
-
-      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
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/jdbc/src/java/org/apache/hive/jdbc/LlapRowInputFormat.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapRowInputFormat.java b/jdbc/src/java/org/apache/hive/jdbc/LlapRowInputFormat.java
new file mode 100644
index 0000000..1cca66a
--- /dev/null
+++ b/jdbc/src/java/org/apache/hive/jdbc/LlapRowInputFormat.java
@@ -0,0 +1,34 @@
+package org.apache.hive.jdbc;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hive.llap.LlapBaseRecordReader;
+import org.apache.hadoop.hive.llap.LlapRowRecordReader;
+import org.apache.hadoop.hive.llap.Row;
+import org.apache.hadoop.hive.llap.Schema;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+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.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+public class LlapRowInputFormat implements InputFormat<NullWritable, Row> {
+  LlapBaseInputFormat<Text> baseInputFormat = new LlapBaseInputFormat<Text>();
+
+  @Override
+  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
+    return baseInputFormat.getSplits(job, numSplits);
+  }
+
+  @Override
+  public RecordReader<NullWritable, Row> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
+      throws IOException {
+    LlapInputSplit<Text> llapSplit = (LlapInputSplit<Text>) split;
+    LlapBaseRecordReader<Text> reader = (LlapBaseRecordReader<Text>) baseInputFormat.getRecordReader(llapSplit, job, reporter);
+    return new LlapRowRecordReader(job, reader.getSchema(), reader);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/llap-common/pom.xml
----------------------------------------------------------------------
diff --git a/llap-common/pom.xml b/llap-common/pom.xml
index 5343479..ceac83b 100644
--- a/llap-common/pom.xml
+++ b/llap-common/pom.xml
@@ -39,6 +39,11 @@
       <artifactId>hive-common</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-serde</artifactId>
+      <version>${project.version}</version>
+    </dependency>
 
     <!-- inter-project -->
     <dependency>
@@ -58,6 +63,22 @@
         </exclusions>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>${hadoop.version}</version>
+      <optional>true</optional>
+            <exclusions>
+            <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commmons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+        </exclusions>
+    </dependency>
+    <dependency>
       <groupId>org.apache.tez</groupId>
       <artifactId>tez-api</artifactId>
       <version>${tez.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/llap-common/src/java/org/apache/hadoop/hive/llap/FieldDesc.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/FieldDesc.java b/llap-common/src/java/org/apache/hadoop/hive/llap/FieldDesc.java
new file mode 100644
index 0000000..9621978
--- /dev/null
+++ b/llap-common/src/java/org/apache/hadoop/hive/llap/FieldDesc.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.llap;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.io.Writable;
+
+public class FieldDesc implements Writable {
+  private String name;
+  private TypeDesc typeDesc;
+
+  public FieldDesc() {
+    typeDesc = new TypeDesc();
+  }
+
+  public FieldDesc(String name, TypeDesc typeDesc) {
+    this.name = name;
+    this.typeDesc = typeDesc;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public TypeDesc getTypeDesc() {
+    return typeDesc;
+  }
+
+  @Override
+  public String toString() {
+    return getName() + ":" + getTypeDesc().toString();
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeUTF(name);
+    typeDesc.write(out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    name = in.readUTF();
+    typeDesc.readFields(in);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/llap-common/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java b/llap-common/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
new file mode 100644
index 0000000..4e000ff
--- /dev/null
+++ b/llap-common/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
@@ -0,0 +1,155 @@
+package org.apache.hadoop.hive.llap;
+
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.JobConf;
+
+import org.apache.hadoop.hive.llap.Row;
+import org.apache.hadoop.hive.llap.FieldDesc;
+import org.apache.hadoop.hive.llap.Schema;
+import org.apache.hadoop.hive.llap.TypeDesc;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
+import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
+import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class LlapRowRecordReader implements RecordReader<NullWritable, Row> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LlapRowRecordReader.class);
+
+  Configuration conf;
+  RecordReader<NullWritable, Text> reader;
+  Schema schema;
+  SerDe serde;
+  final Text textData = new Text();
+
+  public LlapRowRecordReader(Configuration conf, Schema schema, RecordReader<NullWritable, Text> reader) {
+    this.conf = conf;
+    this.schema = schema;
+    this.reader = reader;
+  }
+
+  @Override
+  public void close() throws IOException {
+    reader.close();
+  }
+
+  @Override
+  public NullWritable createKey() {
+    return NullWritable.get();
+  }
+
+  @Override
+  public Row createValue() {
+    return new Row(schema);
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return 0;
+  }
+
+  @Override
+  public float getProgress() throws IOException {
+    return 0;
+  }
+
+  @Override
+  public boolean next(NullWritable key, Row value) throws IOException {
+    Preconditions.checkArgument(value != null);
+
+    if (serde == null) {
+      try {
+        serde = initSerDe(conf);
+      } catch (SerDeException err) {
+        throw new IOException(err);
+      }
+    }
+
+    boolean hasNext = reader.next(key,  textData);
+    if (hasNext) {
+      // Deserialize Text to column values, and populate the row record
+      Object rowObj;
+      try {
+        StructObjectInspector rowOI = (StructObjectInspector) serde.getObjectInspector();
+        rowObj = serde.deserialize(textData);
+        List<? extends StructField> colFields = rowOI.getAllStructFieldRefs();
+        for (int idx = 0; idx < colFields.size(); ++idx) {
+          StructField field = colFields.get(idx);
+          Object colValue = rowOI.getStructFieldData(rowObj, field);
+          Preconditions.checkState(field.getFieldObjectInspector().getCategory() == Category.PRIMITIVE,
+              "Cannot handle non-primitive column type " + field.getFieldObjectInspector().getTypeName());
+
+          PrimitiveObjectInspector poi = (PrimitiveObjectInspector) field.getFieldObjectInspector();
+          // char/varchar special cased here since the row record handles them using Text
+          switch (poi.getPrimitiveCategory()) {
+            case CHAR:
+              value.setValue(idx, ((HiveCharWritable) poi.getPrimitiveWritableObject(colValue)).getPaddedValue());
+              break;
+            case VARCHAR:
+              value.setValue(idx, ((HiveVarcharWritable) poi.getPrimitiveWritableObject(colValue)).getTextValue());
+              break;
+            default:
+              value.setValue(idx, (Writable) poi.getPrimitiveWritableObject(colValue));
+              break;
+          }
+        }
+      } catch (SerDeException err) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Error deserializing row from text: " + textData);
+        }
+        throw new IOException("Error deserializing row data", err);
+      }
+    }
+
+    return hasNext;
+  }
+
+  public Schema getSchema() {
+    return schema;
+  }
+
+  protected SerDe initSerDe(Configuration conf) throws SerDeException {
+    Properties props = new Properties();
+    StringBuffer columnsBuffer = new StringBuffer();
+    StringBuffer typesBuffer = new StringBuffer();
+    boolean isFirst = true;
+    for (FieldDesc colDesc : schema.getColumns()) {
+      if (!isFirst) {
+        columnsBuffer.append(',');
+        typesBuffer.append(',');
+      }
+      columnsBuffer.append(colDesc.getName());
+      typesBuffer.append(colDesc.getTypeDesc().toString());
+      isFirst = false;
+    }
+    String columns = columnsBuffer.toString();
+    String types = typesBuffer.toString();
+    props.put(serdeConstants.LIST_COLUMNS, columns);
+    props.put(serdeConstants.LIST_COLUMN_TYPES, types);
+    SerDe serde = new LazySimpleSerDe();
+    serde.initialize(conf, props);
+
+    return serde;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/llap-common/src/java/org/apache/hadoop/hive/llap/Row.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/Row.java b/llap-common/src/java/org/apache/hadoop/hive/llap/Row.java
new file mode 100644
index 0000000..a84fadc
--- /dev/null
+++ b/llap-common/src/java/org/apache/hadoop/hive/llap/Row.java
@@ -0,0 +1,166 @@
+/**
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hive.serde2.io.ByteWritable;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
+import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+
+public class Row {
+  private final Schema schema;
+  private final Writable[] colValues;
+  private final boolean[] nullIndicators;
+  private Map<String, Integer> nameToIndexMapping;
+
+  public Row(Schema schema) {
+    this.schema = schema;
+    this.colValues = new Writable[schema.getColumns().size()];
+    this.nullIndicators = new boolean[schema.getColumns().size()];
+    this.nameToIndexMapping = new HashMap<String, Integer>(schema.getColumns().size());
+
+    List<FieldDesc> colDescs = schema.getColumns();
+    for (int idx = 0; idx < colDescs.size(); ++idx) {
+      FieldDesc colDesc = colDescs.get(idx);
+      nameToIndexMapping.put(colDesc.getName(), idx);
+      colValues[idx] = createWritableForType(colDesc.getTypeDesc());
+    }
+  }
+
+  public Writable getValue(int colIndex) {
+    if (nullIndicators[colIndex]) {
+      return null;
+    }
+    return colValues[colIndex];
+  }
+
+  public Writable getValue(String colName) {
+    Integer idx = nameToIndexMapping.get(colName);
+    Preconditions.checkArgument(idx != null);
+    return getValue(idx);
+  }
+
+  public Schema getSchema() {
+    return schema;
+  }
+
+  void setValue(int colIdx, Writable value) {
+    Preconditions.checkArgument(colIdx <= schema.getColumns().size());
+
+    if (value == null) {
+      nullIndicators[colIdx] = true;
+    } else {
+      nullIndicators[colIdx] = false;
+      FieldDesc colDesc = schema.getColumns().get(colIdx);
+      switch (colDesc.getTypeDesc().getType()) {
+        case BOOLEAN:
+          ((BooleanWritable) colValues[colIdx]).set(((BooleanWritable) value).get());
+          break;
+        case TINYINT:
+          ((ByteWritable) colValues[colIdx]).set(((ByteWritable) value).get());
+          break;
+        case SMALLINT:
+          ((ShortWritable) colValues[colIdx]).set(((ShortWritable) value).get());
+          break;
+        case INT:
+          ((IntWritable) colValues[colIdx]).set(((IntWritable) value).get());
+          break;
+        case BIGINT:
+          ((LongWritable) colValues[colIdx]).set(((LongWritable) value).get());
+          break;
+        case FLOAT:
+          ((FloatWritable) colValues[colIdx]).set(((FloatWritable) value).get());
+          break;
+        case DOUBLE:
+          ((DoubleWritable) colValues[colIdx]).set(((DoubleWritable) value).get());
+          break;
+        case STRING:
+        // Just handle char/varchar as Text
+        case CHAR:
+        case VARCHAR:
+          ((Text) colValues[colIdx]).set((Text) value);
+          break;
+        case DATE:
+          ((DateWritable) colValues[colIdx]).set((DateWritable) value);
+          break;
+        case TIMESTAMP:
+          ((TimestampWritable) colValues[colIdx]).set((TimestampWritable) value);
+          break;
+        case BINARY:
+          ((BytesWritable) colValues[colIdx]).set(((BytesWritable) value));
+          break;
+        case DECIMAL:
+          ((HiveDecimalWritable) colValues[colIdx]).set((HiveDecimalWritable) value);
+          break;
+      }
+    }
+  }
+
+  private Writable createWritableForType(TypeDesc typeDesc) {
+    switch (typeDesc.getType()) {
+      case BOOLEAN:
+        return new BooleanWritable();
+      case TINYINT:
+        return new ByteWritable();
+      case SMALLINT:
+        return new ShortWritable();
+      case INT:
+        return new IntWritable();
+      case BIGINT:
+        return new LongWritable();
+      case FLOAT:
+        return new FloatWritable();
+      case DOUBLE:
+        return new DoubleWritable();
+      case STRING:
+      // Just handle char/varchar as Text
+      case CHAR:
+      case VARCHAR:
+        return new Text();
+      case DATE:
+        return new DateWritable();
+      case TIMESTAMP:
+        return new TimestampWritable();
+      case BINARY:
+        return new BytesWritable();
+      case DECIMAL:
+        return new HiveDecimalWritable();
+      default:
+        throw new RuntimeException("Cannot create writable for " + typeDesc.getType());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/llap-common/src/java/org/apache/hadoop/hive/llap/Schema.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/Schema.java b/llap-common/src/java/org/apache/hadoop/hive/llap/Schema.java
new file mode 100644
index 0000000..c1bf4ea
--- /dev/null
+++ b/llap-common/src/java/org/apache/hadoop/hive/llap/Schema.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.llap;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.io.Writable;
+
+public class Schema implements Writable {
+
+  private final List<FieldDesc> columns;
+
+  public Schema(List<FieldDesc> columns) {
+    this.columns = columns;
+  }
+
+  public Schema() {
+    columns = new ArrayList<FieldDesc>();
+  }
+
+  public List<FieldDesc> getColumns() {
+    return columns;
+  }
+
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer();
+    boolean first = true;
+    for (FieldDesc colDesc : getColumns()) {
+      if (!first) {
+        sb.append(",");
+      }
+      sb.append(colDesc.toString());
+      first = false;
+    }
+    return sb.toString();
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(columns.size());
+    for (FieldDesc column : columns) {
+      column.write(out);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    int numColumns = in.readInt();
+    columns.clear();
+    for (int idx = 0; idx < numColumns; ++idx) {
+      FieldDesc colDesc = new FieldDesc();
+      colDesc.readFields(in);
+      columns.add(colDesc);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/llap-common/src/java/org/apache/hadoop/hive/llap/TypeDesc.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/TypeDesc.java b/llap-common/src/java/org/apache/hadoop/hive/llap/TypeDesc.java
new file mode 100644
index 0000000..dda5928
--- /dev/null
+++ b/llap-common/src/java/org/apache/hadoop/hive/llap/TypeDesc.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.llap;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.io.Writable;
+
+public class TypeDesc implements Writable {
+  public static enum Type {
+    BOOLEAN,
+    TINYINT,
+    SMALLINT,
+    INT,
+    BIGINT,
+    FLOAT,
+    DOUBLE,
+    STRING,
+    CHAR,
+    VARCHAR,
+    DATE,
+    TIMESTAMP,
+    BINARY,
+    DECIMAL,
+  }
+
+  private TypeDesc.Type type;
+  private int precision;
+  private int scale;
+
+  // For types with no type qualifiers
+  public TypeDesc(TypeDesc.Type type) {
+    this(type, 0, 0);
+  }
+
+  // For decimal types
+  public TypeDesc(TypeDesc.Type type, int precision, int scale) {
+    this.type = type;
+    this.precision = precision;
+    this.scale = scale;
+  }
+
+  // For char/varchar types
+  public TypeDesc(TypeDesc.Type type, int precision) {
+    this(type, precision, 0);
+  }
+
+  // Should be used for serialization only
+  public TypeDesc() {
+    this(TypeDesc.Type.INT, 0, 0);
+  }
+
+  public TypeDesc.Type getType() {
+    return type;
+  }
+
+  public int getPrecision() {
+    return precision;
+  }
+
+  public int getScale() {
+    return scale;
+  }
+
+  @Override
+  public String toString() {
+    switch (type) {
+      case DECIMAL:
+        return type.name().toLowerCase() + "(" + precision + "," + scale + ")";
+      case CHAR:
+      case VARCHAR:
+        return type.name().toLowerCase() + "(" + precision + ")";
+      default:
+        return type.name().toLowerCase();
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeUTF(type.name());
+    out.writeInt(precision);
+    out.writeInt(scale);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    type = TypeDesc.Type.valueOf(in.readUTF());
+    precision = in.readInt();
+    scale = in.readInt();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/llap-common/src/test/org/apache/hadoop/hive/llap/TestRow.java
----------------------------------------------------------------------
diff --git a/llap-common/src/test/org/apache/hadoop/hive/llap/TestRow.java b/llap-common/src/test/org/apache/hadoop/hive/llap/TestRow.java
new file mode 100644
index 0000000..d4e68f4
--- /dev/null
+++ b/llap-common/src/test/org/apache/hadoop/hive/llap/TestRow.java
@@ -0,0 +1,92 @@
+/*
+ * 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.Random;
+
+import org.apache.commons.lang.RandomStringUtils;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestRow {
+
+  @Test
+  public void testUsage() {
+    Schema schema = createTestSchema();
+    Row row = new Row(schema);
+
+    Random rand = new Random();
+    int iterations = 100;
+    Text col0 = new Text();
+    IntWritable col1 = new IntWritable();
+    for (int idx = 0; idx < iterations; ++idx) {
+      // Set the row values
+      boolean isNullCol0 = (rand.nextDouble() <= 0.25);
+      col0.set(RandomStringUtils.random(10));
+      row.setValue(0, isNullCol0 ? null : col0);
+
+      boolean isNullCol1 = (rand.nextDouble() <= 0.25);
+      col1.set(rand.nextInt());
+      row.setValue(1, isNullCol1 ? null : col1);
+
+      // Validate the row values
+      if (isNullCol0) {
+        assertTrue(row.getValue(0) == null);
+        assertTrue(row.getValue("col0") == null);
+      } else {
+        assertTrue(row.getValue(0) != null);
+        assertTrue(col0 != row.getValue(0));
+        assertEquals(col0, row.getValue(0));
+        assertEquals(col0, row.getValue("col0"));
+      }
+
+      if (isNullCol1) {
+        assertTrue(row.getValue(1) == null);
+        assertTrue(row.getValue("col1") == null);
+      } else {
+        assertTrue(row.getValue(1) != null);
+        assertTrue(col1 != row.getValue(1));
+        assertEquals(col1, row.getValue(1));
+        assertEquals(col1, row.getValue("col1"));
+      }
+    }
+  }
+
+  private Schema createTestSchema() {
+    List<FieldDesc> colDescs = new ArrayList<FieldDesc>();
+
+    colDescs.add(new FieldDesc("col0",
+        new TypeDesc(TypeDesc.Type.STRING)));
+
+    colDescs.add(new FieldDesc("col1",
+        new TypeDesc(TypeDesc.Type.INT)));
+
+    Schema schema = new Schema(colDescs);
+    return schema;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
index aaca7d6..0930d60 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
@@ -30,7 +30,7 @@ import com.google.protobuf.ByteString;
 
 import org.apache.commons.collections4.ListUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.llap.LlapRecordReader.ReaderEvent;
+import org.apache.hadoop.hive.llap.LlapBaseRecordReader.ReaderEvent;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
 import org.apache.hadoop.hive.llap.ext.LlapTaskUmbilicalExternalClient;
@@ -74,7 +74,6 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
 
   private static final Logger LOG = LoggerFactory.getLogger(LlapInputFormat.class);
 
-
   public LlapInputFormat() {
   }
 
@@ -135,7 +134,7 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
 
     LOG.info("Registered id: " + id);
 
-    LlapRecordReader recordReader = new LlapRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class);
+    LlapBaseRecordReader recordReader = new LlapBaseRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class);
     umbilicalResponder.setRecordReader(recordReader);
     return recordReader;
   }
@@ -276,7 +275,7 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
   }
 
   private static class LlapRecordReaderTaskUmbilicalExternalResponder implements LlapTaskUmbilicalExternalResponder {
-    protected LlapRecordReader recordReader = null;
+    protected LlapBaseRecordReader recordReader = null;
     protected LinkedBlockingQueue<ReaderEvent> queuedEvents = new LinkedBlockingQueue<ReaderEvent>();
 
     public LlapRecordReaderTaskUmbilicalExternalResponder() {
@@ -285,7 +284,7 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     @Override
     public void submissionFailed(String fragmentId, Throwable throwable) {
       try {
-        sendOrQueueEvent(LlapRecordReader.ReaderEvent.errorEvent(
+        sendOrQueueEvent(ReaderEvent.errorEvent(
             "Received submission failed event for fragment ID " + fragmentId));
       } catch (Exception err) {
         LOG.error("Error during heartbeat responder:", err);
@@ -301,11 +300,11 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
         try {
           switch (eventType) {
             case TASK_ATTEMPT_COMPLETED_EVENT:
-              sendOrQueueEvent(LlapRecordReader.ReaderEvent.doneEvent());
+              sendOrQueueEvent(ReaderEvent.doneEvent());
               break;
             case TASK_ATTEMPT_FAILED_EVENT:
               TaskAttemptFailedEvent taskFailedEvent = (TaskAttemptFailedEvent) tezEvent.getEvent();
-              sendOrQueueEvent(LlapRecordReader.ReaderEvent.errorEvent(taskFailedEvent.getDiagnostics()));
+              sendOrQueueEvent(ReaderEvent.errorEvent(taskFailedEvent.getDiagnostics()));
               break;
             case TASK_STATUS_UPDATE_EVENT:
               // If we want to handle counters
@@ -323,7 +322,7 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     @Override
     public void taskKilled(TezTaskAttemptID taskAttemptId) {
       try {
-        sendOrQueueEvent(LlapRecordReader.ReaderEvent.errorEvent(
+        sendOrQueueEvent(ReaderEvent.errorEvent(
             "Received task killed event for task ID " + taskAttemptId));
       } catch (Exception err) {
         LOG.error("Error during heartbeat responder:", err);
@@ -333,18 +332,18 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     @Override
     public void heartbeatTimeout(String taskAttemptId) {
       try {
-        sendOrQueueEvent(LlapRecordReader.ReaderEvent.errorEvent(
+        sendOrQueueEvent(ReaderEvent.errorEvent(
             "Timed out waiting for heartbeat for task ID " + taskAttemptId));
       } catch (Exception err) {
         LOG.error("Error during heartbeat responder:", err);
       }
     }
 
-    public synchronized LlapRecordReader getRecordReader() {
+    public synchronized LlapBaseRecordReader getRecordReader() {
       return recordReader;
     }
 
-    public synchronized void setRecordReader(LlapRecordReader recordReader) {
+    public synchronized void setRecordReader(LlapBaseRecordReader recordReader) {
       this.recordReader = recordReader;
 
       if (recordReader == null) {
@@ -353,7 +352,7 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
 
       // If any events were queued by the responder, give them to the record reader now.
       while (!queuedEvents.isEmpty()) {
-        LlapRecordReader.ReaderEvent readerEvent = queuedEvents.poll();
+        ReaderEvent readerEvent = queuedEvents.poll();
         LOG.debug("Sending queued event to record reader: " + readerEvent.getEventType());
         recordReader.handleEvent(readerEvent);
       }
@@ -365,8 +364,8 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
      * since we don't want to drop these events.
      * @param readerEvent
      */
-    protected synchronized void sendOrQueueEvent(LlapRecordReader.ReaderEvent readerEvent) {
-      LlapRecordReader recordReader = getRecordReader();
+    protected synchronized void sendOrQueueEvent(ReaderEvent readerEvent) {
+      LlapBaseRecordReader recordReader = getRecordReader();
       if (recordReader != null) {
         recordReader.handleEvent(readerEvent);
       } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/ql/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
new file mode 100644
index 0000000..7073280
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
@@ -0,0 +1,205 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.llap;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.DataInputStream;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.llap.Schema;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.JobConf;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LlapBaseRecordReader<V extends WritableComparable> implements RecordReader<NullWritable, V> {
+  private static final Logger LOG = LoggerFactory.getLogger(LlapBaseRecordReader.class);
+
+  DataInputStream din;
+  Schema schema;
+  Class<V> clazz;
+
+
+  protected Thread readerThread = null;
+  protected LinkedBlockingQueue<ReaderEvent> readerEvents = new LinkedBlockingQueue<ReaderEvent>();
+
+  public LlapBaseRecordReader(InputStream in, Schema schema, Class<V> clazz) {
+    din = new DataInputStream(in);
+    this.schema = schema;
+    this.clazz = clazz;
+    this.readerThread = Thread.currentThread();
+  }
+
+  public Schema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public void close() throws IOException {
+    din.close();
+  }
+
+  @Override
+  public long getPos() { return 0; }
+
+  @Override
+  public float getProgress() { return 0f; }
+
+  @Override
+  public NullWritable createKey() {
+    return NullWritable.get();
+  }
+
+  @Override
+  public V createValue() {
+    try {
+      return clazz.newInstance();
+    } catch (Exception e) {
+      return null;
+    }
+  }
+
+  @Override
+  public boolean next(NullWritable key, V value) throws IOException {
+    try {
+      // Need a way to know what thread to interrupt, since this is a blocking thread.
+      setReaderThread(Thread.currentThread());
+
+      value.readFields(din);
+      return true;
+    } catch (EOFException eof) {
+      // End of input. There should be a reader event available, or coming soon, so okay to be blocking call.
+      ReaderEvent event = getReaderEvent();
+      switch (event.getEventType()) {
+        case DONE:
+          break;
+        default:
+          throw new IOException("Expected reader event with done status, but got "
+              + event.getEventType() + " with message " + event.getMessage());
+      }
+      return false;
+    } catch (IOException io) {
+      if (Thread.interrupted()) {
+        // Either we were interrupted by one of:
+        // 1. handleEvent(), in which case there is a reader event waiting for us in the queue
+        // 2. Some other unrelated cause which interrupted us, in which case there may not be a reader event coming.
+        // Either way we should not try to block trying to read the reader events queue.
+        if (readerEvents.isEmpty()) {
+          // Case 2.
+          throw io;
+        } else {
+          // Case 1. Fail the reader, sending back the error we received from the reader event.
+          ReaderEvent event = getReaderEvent();
+          switch (event.getEventType()) {
+            case ERROR:
+              throw new IOException("Received reader event error: " + event.getMessage());
+            default:
+              throw new IOException("Got reader event type " + event.getEventType() + ", expected error event");
+          }
+        }
+      } else {
+        // If we weren't interrupted, just propagate the error
+        throw io;
+      }
+    }
+  }
+
+  /**
+   * Define success/error events which are passed to the reader from a different thread.
+   * The reader will check for these events on end of input and interruption of the reader thread.
+   */
+  public static class ReaderEvent {
+    public enum EventType {
+      DONE,
+      ERROR
+    }
+
+    protected final EventType eventType;
+    protected final String message;
+
+    protected ReaderEvent(EventType type, String message) {
+      this.eventType = type;
+      this.message = message;
+    }
+
+    public static ReaderEvent doneEvent() {
+      return new ReaderEvent(EventType.DONE, "");
+    }
+
+    public static ReaderEvent errorEvent(String message) {
+      return new ReaderEvent(EventType.ERROR, message);
+    }
+
+    public EventType getEventType() {
+      return eventType;
+    }
+
+    public String getMessage() {
+      return message;
+    }
+  }
+
+  public void handleEvent(ReaderEvent event) {
+    switch (event.getEventType()) {
+      case DONE:
+        // Reader will check for the event queue upon the end of the input stream - no need to interrupt.
+        readerEvents.add(event);
+        break;
+      case ERROR:
+        readerEvents.add(event);
+        if (readerThread == null) {
+          throw new RuntimeException("Reader thread is unexpectedly null, during ReaderEvent error " + event.getMessage());
+        }
+        // Reader is using a blocking socket .. interrupt it.
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Interrupting reader thread due to reader event with error " + event.getMessage());
+        }
+        getReaderThread().interrupt();
+        break;
+      default:
+        throw new RuntimeException("Unhandled ReaderEvent type " + event.getEventType() + " with message " + event.getMessage());
+    }
+  }
+
+  protected ReaderEvent getReaderEvent() {
+    try {
+      ReaderEvent event = readerEvents.take();
+      return event;
+    } catch (InterruptedException ie) {
+      throw new RuntimeException("Interrupted while getting readerEvents, not expected", ie);
+    }
+  }
+
+  protected synchronized void setReaderThread(Thread readerThread) {
+    this.readerThread = readerThread;
+  }
+
+  protected synchronized Thread getReaderThread() {
+    return readerThread;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
index 17a0d2d..02aedfd 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
@@ -20,7 +20,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.llap.Schema;
 import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
 import org.apache.hadoop.mapred.SplitLocationInfo;
 import org.apache.thrift.TDeserializer;
@@ -93,17 +93,7 @@ public class LlapInputSplit implements InputSplitWithLocationInfo {
       out.writeUTF(locations[i].getLocation());
     }
 
-    byte[] binarySchema;
-
-    try {
-      TSerializer serializer = new TSerializer();
-      byte[] serialzied = serializer.serialize(schema);
-      out.writeInt(serialzied.length);
-      out.write(serialzied);
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-
+    schema.write(out);
     out.writeUTF(llapUser);
   }
 
@@ -125,17 +115,8 @@ public class LlapInputSplit implements InputSplitWithLocationInfo {
       locations[i] = new SplitLocationInfo(in.readUTF(), false);
     }
 
-    length = in.readInt();
-
-    try {
-      byte[] schemaBytes = new byte[length];
-      in.readFully(schemaBytes);
-      TDeserializer tDeserializer = new TDeserializer();
-      schema = new Schema();
-      tDeserializer.deserialize(schema, schemaBytes);
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
+    schema = new Schema();
+    schema.readFields(in);
     llapUser = in.readUTF();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
deleted file mode 100644
index 64e5e69..0000000
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.llap;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.DataInputStream;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.ql.io.RCFile.Reader;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.hive.metastore.api.Schema;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class LlapRecordReader<V extends WritableComparable> implements RecordReader<NullWritable, V> {
-  private static final Logger LOG = LoggerFactory.getLogger(LlapRecordReader.class);
-
-  DataInputStream din;
-  Schema schema;
-  Class<V> clazz;
-
-
-  protected Thread readerThread = null;
-  protected LinkedBlockingQueue<ReaderEvent> readerEvents = new LinkedBlockingQueue<ReaderEvent>();
-
-  public LlapRecordReader(InputStream in, Schema schema, Class<V> clazz) {
-    din = new DataInputStream(in);
-    this.schema = schema;
-    this.clazz = clazz;
-    this.readerThread = Thread.currentThread();
-  }
-
-  public Schema getSchema() {
-    return schema;
-  }
-
-  @Override
-  public void close() throws IOException {
-    din.close();
-  }
-
-  @Override
-  public long getPos() { return 0; }
-
-  @Override
-  public float getProgress() { return 0f; }
-
-  @Override
-  public NullWritable createKey() {
-    return NullWritable.get();
-  }
-
-  @Override
-  public V createValue() {
-    try {
-      return clazz.newInstance();
-    } catch (Exception e) {
-      return null;
-    }
-  }
-
-  @Override
-  public boolean next(NullWritable key, V value) throws IOException {
-    try {
-      // Need a way to know what thread to interrupt, since this is a blocking thread.
-      setReaderThread(Thread.currentThread());
-
-      value.readFields(din);
-      return true;
-    } catch (EOFException eof) {
-      // End of input. There should be a reader event available, or coming soon, so okay to be blocking call.
-      ReaderEvent event = getReaderEvent();
-      switch (event.getEventType()) {
-        case DONE:
-          break;
-        default:
-          throw new IOException("Expected reader event with done status, but got "
-              + event.getEventType() + " with message " + event.getMessage());
-      }
-      return false;
-    } catch (IOException io) {
-      if (Thread.interrupted()) {
-        // Either we were interrupted by one of:
-        // 1. handleEvent(), in which case there is a reader event waiting for us in the queue
-        // 2. Some other unrelated cause which interrupted us, in which case there may not be a reader event coming.
-        // Either way we should not try to block trying to read the reader events queue.
-        if (readerEvents.isEmpty()) {
-          // Case 2.
-          throw io;
-        } else {
-          // Case 1. Fail the reader, sending back the error we received from the reader event.
-          ReaderEvent event = getReaderEvent();
-          switch (event.getEventType()) {
-            case ERROR:
-              throw new IOException("Received reader event error: " + event.getMessage());
-            default:
-              throw new IOException("Got reader event type " + event.getEventType() + ", expected error event");
-          }
-        }
-      } else {
-        // If we weren't interrupted, just propagate the error
-        throw io;
-      }
-    }
-  }
-
-  /**
-   * Define success/error events which are passed to the reader from a different thread.
-   * The reader will check for these events on end of input and interruption of the reader thread.
-   */
-  public static class ReaderEvent {
-    public enum EventType {
-      DONE,
-      ERROR
-    }
-
-    protected final EventType eventType;
-    protected final String message;
-
-    protected ReaderEvent(EventType type, String message) {
-      this.eventType = type;
-      this.message = message;
-    }
-
-    public static ReaderEvent doneEvent() {
-      return new ReaderEvent(EventType.DONE, "");
-    }
-
-    public static ReaderEvent errorEvent(String message) {
-      return new ReaderEvent(EventType.ERROR, message);
-    }
-
-    public EventType getEventType() {
-      return eventType;
-    }
-
-    public String getMessage() {
-      return message;
-    }
-  }
-
-  public void handleEvent(ReaderEvent event) {
-    switch (event.getEventType()) {
-      case DONE:
-        // Reader will check for the event queue upon the end of the input stream - no need to interrupt.
-        readerEvents.add(event);
-        break;
-      case ERROR:
-        readerEvents.add(event);
-        if (readerThread == null) {
-          throw new RuntimeException("Reader thread is unexpectedly null, during ReaderEvent error " + event.getMessage());
-        }
-        // Reader is using a blocking socket .. interrupt it.
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Interrupting reader thread due to reader event with error " + event.getMessage());
-        }
-        getReaderThread().interrupt();
-        break;
-      default:
-        throw new RuntimeException("Unhandled ReaderEvent type " + event.getEventType() + " with message " + event.getMessage());
-    }
-  }
-
-  protected ReaderEvent getReaderEvent() {
-    try {
-      ReaderEvent event = readerEvents.take();
-      return event;
-    } catch (InterruptedException ie) {
-      throw new RuntimeException("Interrupted while getting readerEvents, not expected", ie);
-    }
-  }
-
-  protected synchronized void setReaderThread(Thread readerThread) {
-    this.readerThread = readerThread;
-  }
-
-  protected synchronized Thread getReaderThread() {
-    return readerThread;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
index 6267324..51027a7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
@@ -45,7 +45,10 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.LlapInputSplit;
 import org.apache.hadoop.hive.llap.LlapOutputFormat;
 import org.apache.hadoop.hive.llap.SubmitWorkInfo;
-import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.llap.Schema;
+import org.apache.hadoop.hive.llap.FieldDesc;
+import org.apache.hadoop.hive.llap.TypeDesc;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.Driver;
@@ -71,6 +74,12 @@ import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
@@ -220,7 +229,7 @@ public class GenericUDTFGetSplits extends GenericUDTF {
 
     QueryPlan plan = driver.getPlan();
     List<Task<?>> roots = plan.getRootTasks();
-    Schema schema = plan.getResultSchema();
+    Schema schema = convertSchema(plan.getResultSchema());
 
     if (roots == null || roots.size() != 1 || !(roots.get(0) instanceof TezTask)) {
       throw new HiveException("Was expecting a single TezTask.");
@@ -255,7 +264,7 @@ public class GenericUDTFGetSplits extends GenericUDTF {
 
       plan = driver.getPlan();
       roots = plan.getRootTasks();
-      schema = plan.getResultSchema();
+      schema = convertSchema(plan.getResultSchema());
 
       if (roots == null || roots.size() != 1 || !(roots.get(0) instanceof TezTask)) {
         throw new HiveException("Was expecting a single TezTask.");
@@ -416,6 +425,78 @@ public class GenericUDTFGetSplits extends GenericUDTF {
     }
   }
 
+  private TypeDesc convertTypeString(String typeString) throws HiveException {
+    TypeDesc typeDesc;
+    TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(typeString);
+    Preconditions.checkState(typeInfo.getCategory() == ObjectInspector.Category.PRIMITIVE,
+        "Unsupported non-primitive type " + typeString);
+
+    switch (((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory()) {
+      case BOOLEAN:
+        typeDesc = new TypeDesc(TypeDesc.Type.BOOLEAN);
+        break;
+      case BYTE:
+        typeDesc = new TypeDesc(TypeDesc.Type.TINYINT);
+        break;
+      case SHORT:
+        typeDesc = new TypeDesc(TypeDesc.Type.SMALLINT);
+        break;
+      case INT:
+        typeDesc = new TypeDesc(TypeDesc.Type.INT);
+        break;
+      case LONG:
+        typeDesc = new TypeDesc(TypeDesc.Type.BIGINT);
+        break;
+      case FLOAT:
+        typeDesc = new TypeDesc(TypeDesc.Type.FLOAT);
+        break;
+      case DOUBLE:
+        typeDesc = new TypeDesc(TypeDesc.Type.DOUBLE);
+        break;
+      case STRING:
+        typeDesc = new TypeDesc(TypeDesc.Type.STRING);
+        break;
+      case CHAR:
+        CharTypeInfo charTypeInfo = (CharTypeInfo) typeInfo;
+        typeDesc = new TypeDesc(TypeDesc.Type.CHAR, charTypeInfo.getLength());
+        break;
+      case VARCHAR:
+        VarcharTypeInfo varcharTypeInfo = (VarcharTypeInfo) typeInfo;
+        typeDesc = new TypeDesc(TypeDesc.Type.CHAR, varcharTypeInfo.getLength());
+        break;
+      case DATE:
+        typeDesc = new TypeDesc(TypeDesc.Type.DATE);
+        break;
+      case TIMESTAMP:
+        typeDesc = new TypeDesc(TypeDesc.Type.TIMESTAMP);
+        break;
+      case BINARY:
+        typeDesc = new TypeDesc(TypeDesc.Type.BINARY);
+        break;
+      case DECIMAL:
+        DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfo;
+        typeDesc = new TypeDesc(TypeDesc.Type.DECIMAL, decimalTypeInfo.getPrecision(), decimalTypeInfo.getScale());
+        break;
+      default:
+        throw new HiveException("Unsupported type " + typeString);
+    }
+
+    return typeDesc;
+  }
+
+  private Schema convertSchema(Object obj) throws HiveException {
+    org.apache.hadoop.hive.metastore.api.Schema schema = (org.apache.hadoop.hive.metastore.api.Schema) obj;
+    List<FieldDesc> colDescs = new ArrayList<FieldDesc>();
+    for (FieldSchema fs : schema.getFieldSchemas()) {
+      String colName = fs.getName();
+      String typeString = fs.getType();
+      TypeDesc typeDesc = convertTypeString(typeString);
+      colDescs.add(new FieldDesc(colName, typeDesc));
+    }
+    Schema Schema = new Schema(colDescs);
+    return Schema;
+  }
+
   @Override
   public void close() throws HiveException {
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/fc7343dd/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
index 7b516fe..37e21b8 100644
--- a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.hive.metastore.api.Schema;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.llap.LlapBaseRecordReader.ReaderEvent;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
 
 
@@ -112,10 +113,13 @@ public class TestLlapOutputFormat {
       writer.close(null);
 
       InputStream in = socket.getInputStream();
-      RecordReader reader = new LlapRecordReader(in, null, Text.class);
+      LlapBaseRecordReader reader = new LlapBaseRecordReader(in, null, Text.class);
 
       LOG.debug("Have record reader");
 
+      // Send done event, which LlapRecordReader is expecting upon end of input
+      reader.handleEvent(ReaderEvent.doneEvent());
+
       int count = 0;
       while(reader.next(NullWritable.get(), text)) {
         LOG.debug(text.toString());


[07/39] hive git commit: HIVE-13172: Merge master into llap branch

Posted by jd...@apache.org.
HIVE-13172: Merge master into llap branch


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f6be2180
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f6be2180
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f6be2180

Branch: refs/heads/master
Commit: f6be218067855bb0e6893d71531bfd9c3ce835f6
Parents: 57761e3 157d825
Author: Jason Dere <jd...@hortonworks.com>
Authored: Fri Feb 26 15:08:45 2016 -0800
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Fri Feb 26 15:08:45 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   11 +-
 .../streaming/mutate/StreamingTestUtils.java    |    9 +-
 .../streaming/mutate/TestMutations.java         |    6 +-
 .../hive/metastore/TestHiveMetaStore.java       |  128 +-
 .../hive/ql/txn/compactor/TestCompactor.java    |    3 +-
 itests/qtest/pom.xml                            |    4 +-
 .../test/resources/testconfiguration.properties |    6 +-
 .../hive/llap/registry/ServiceInstanceSet.java  |    7 -
 .../ServiceInstanceStateChangeListener.java     |   42 +
 .../hive/llap/registry/ServiceRegistry.java     |   27 +-
 .../registry/impl/LlapFixedRegistryImpl.java    |   18 +-
 .../llap/registry/impl/LlapRegistryService.java |    8 +-
 .../registry/impl/LlapYarnRegistryImpl.java     |  438 -
 .../impl/LlapZookeeperRegistryImpl.java         |  672 ++
 .../hive/llap/tez/LlapProtocolClientProxy.java  |    6 +
 .../hive/llap/security/LlapTokenIdentifier.java |    2 +-
 llap-server/bin/runLlapDaemon.sh                |    2 +-
 .../hadoop/hive/llap/cli/LlapServiceDriver.java |   38 +-
 .../configuration/LlapDaemonConfiguration.java  |   12 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |    8 +
 .../daemon/impl/LlapProtocolServerImpl.java     |    3 +
 .../llap/daemon/services/impl/LlapWebApp.java   |   29 -
 .../daemon/services/impl/LlapWebServices.java   |   11 +-
 .../hive/llap/security/LlapSecurityHelper.java  |   24 +-
 .../main/resources/hive-webapps/llap/index.html |   72 +-
 .../resources/hive-webapps/llap/js/metrics.js   |  279 +-
 .../llap/tezplugins/LlapTaskCommunicator.java   |    3 +
 .../tezplugins/LlapTaskSchedulerService.java    |   71 +-
 .../apache/hadoop/hive/metastore/Deadline.java  |   63 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |    1 +
 .../hadoop/hive/metastore/RawStoreProxy.java    |   28 +-
 .../hive/metastore/RetryingHMSHandler.java      |   12 +-
 .../metastore/SessionPropertiesListener.java    |    9 +-
 .../TransactionalValidationListener.java        |  179 +
 packaging/pom.xml                               |   43 +
 packaging/src/deb/control/control               |   23 +
 packaging/src/main/assembly/bin.xml             |   10 +
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   15 +
 .../hadoop/hive/ql/exec/FunctionTask.java       |   10 +-
 .../hadoop/hive/ql/exec/MapJoinOperator.java    |    7 +
 .../apache/hadoop/hive/ql/exec/Registry.java    |   44 +
 .../hadoop/hive/ql/exec/tez/DagUtils.java       |    5 +-
 .../hive/ql/exec/tez/TezSessionState.java       |    6 +-
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |    7 +-
 .../hadoop/hive/ql/io/NullScanFileSystem.java   |   15 +
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |  117 +-
 .../ql/optimizer/calcite/HiveCalciteUtil.java   |  110 +-
 .../hive/ql/optimizer/calcite/HiveRexUtil.java  |   78 +-
 .../rules/HiveFilterProjectTransposeRule.java   |  152 +-
 .../HiveJoinPushTransitivePredicatesRule.java   |    2 +-
 .../rules/HiveReduceExpressionsRule.java        |   12 +-
 .../calcite/stats/HiveRelMdPredicates.java      |   40 +-
 .../hive/ql/optimizer/physical/LlapDecider.java |   77 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |    3 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |    3 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    6 +-
 .../apache/hadoop/hive/ql/plan/PlanUtils.java   |    3 +
 .../org/apache/hadoop/hive/ql/plan/TezWork.java |    6 +-
 .../hive/ql/io/orc/TestInputOutputFormat.java   |  225 +-
 .../queries/clientnegative/create_not_acid.q    |    6 +
 .../clientnegative/delete_not_bucketed.q        |    7 -
 .../clientnegative/orc_change_fileformat_acid.q |    2 +-
 .../clientnegative/orc_change_serde_acid.q      |    2 +-
 .../clientnegative/orc_reorder_columns1_acid.q  |    2 +-
 .../clientnegative/orc_reorder_columns2_acid.q  |    2 +-
 .../clientnegative/orc_replace_columns1_acid.q  |    2 +-
 .../clientnegative/orc_replace_columns2_acid.q  |    2 +-
 .../clientnegative/orc_replace_columns3_acid.q  |    2 +-
 .../clientnegative/orc_type_promotion1_acid.q   |    2 +-
 .../clientnegative/orc_type_promotion2_acid.q   |    2 +-
 .../clientnegative/orc_type_promotion3_acid.q   |    2 +-
 .../clientnegative/update_not_bucketed.q        |    7 -
 ql/src/test/queries/clientpositive/constprog3.q |    8 +
 ql/src/test/queries/clientpositive/fold_case.q  |    3 +
 .../queries/clientpositive/infer_join_preds.q   |   61 +
 .../clientpositive/insert_overwrite_directory.q |    8 +-
 ql/src/test/queries/clientpositive/llap_udf.q   |   47 +
 .../test/queries/clientpositive/llapdecider.q   |    4 +
 .../queries/clientpositive/ppd_windowing1.q     |   46 +
 .../queries/clientpositive/ppd_windowing2.q     |   49 +
 .../clientnegative/create_not_acid.q.out        |    5 +
 .../clientnegative/delete_not_bucketed.q.out    |    9 -
 .../orc_change_fileformat_acid.q.out            |    4 +-
 .../clientnegative/orc_change_serde_acid.q.out  |    4 +-
 .../orc_reorder_columns1_acid.q.out             |    4 +-
 .../orc_reorder_columns2_acid.q.out             |    4 +-
 .../orc_replace_columns1_acid.q.out             |    4 +-
 .../orc_replace_columns2_acid.q.out             |    4 +-
 .../orc_replace_columns3_acid.q.out             |    4 +-
 .../orc_type_promotion1_acid.q.out              |    4 +-
 .../orc_type_promotion2_acid.q.out              |    4 +-
 .../orc_type_promotion3_acid.q.out              |    4 +-
 .../clientnegative/update_not_bucketed.q.out    |    9 -
 .../results/clientpositive/constprog3.q.out     |   83 +
 .../clientpositive/correlationoptimizer12.q.out |   68 +-
 .../test/results/clientpositive/fold_case.q.out |   49 +-
 .../clientpositive/infer_join_preds.q.out       |  619 ++
 .../insert_overwrite_directory.q.out            |   24 +-
 .../results/clientpositive/llap/cte_mat_1.q.out |    1 +
 .../results/clientpositive/llap/cte_mat_2.q.out |    1 +
 .../results/clientpositive/llap/cte_mat_3.q.out |    1 +
 .../results/clientpositive/llap/llap_udf.q.out  |  307 +
 .../clientpositive/llap/llapdecider.q.out       |   61 +
 .../test/results/clientpositive/llap/mrr.q.out  |    1 +
 .../results/clientpositive/llap/tez_dml.q.out   |    1 +
 .../llap/tez_dynpart_hashjoin_1.q.out           |    1 +
 .../llap/tez_dynpart_hashjoin_2.q.out           |    1 +
 .../clientpositive/llap/tez_join_tests.q.out    |    1 +
 .../clientpositive/llap/tez_joins_explain.q.out |    1 +
 .../clientpositive/llap/tez_smb_main.q.out      |    1 +
 .../results/clientpositive/llap/tez_union.q.out |    1 +
 .../clientpositive/llap/tez_union2.q.out        |    1 +
 .../llap/tez_vector_dynpart_hashjoin_1.q.out    |    1 +
 .../llap/tez_vector_dynpart_hashjoin_2.q.out    |    1 +
 .../test/results/clientpositive/mergejoin.q.out |    2 -
 .../clientpositive/partition_boolexpr.q.out     |   26 +-
 .../results/clientpositive/perf/query70.q.out   |    4 +-
 .../results/clientpositive/ppd_udf_col.q.out    |   20 +-
 .../results/clientpositive/ppd_windowing1.q.out | 2029 ++++
 .../results/clientpositive/ppd_windowing2.q.out | 9841 ++++++++++++++++++
 .../results/clientpositive/ptfgroupbyjoin.q.out |   66 +-
 .../clientpositive/spark/subquery_in.q.out      |   15 +-
 .../spark/vectorization_short_regress.q.out     |    4 +-
 .../results/clientpositive/subquery_in.q.out    |   15 +-
 .../subquery_unqualcolumnrefs.q.out             |   30 +-
 .../results/clientpositive/tez/auto_join0.q.out |    1 +
 .../clientpositive/tez/auto_join21.q.out        |    1 +
 .../clientpositive/tez/auto_join29.q.out        |    1 +
 .../clientpositive/tez/auto_join30.q.out        |    1 +
 .../tez/correlationoptimizer1.q.out             |    1 +
 .../results/clientpositive/tez/cross_join.q.out |    1 +
 .../results/clientpositive/tez/having.q.out     |    1 +
 .../clientpositive/tez/join0.q.java1.7.out      |    1 +
 .../clientpositive/tez/limit_pushdown.q.out     |    1 +
 .../clientpositive/tez/llapdecider.q.out        |   61 +
 .../clientpositive/tez/mapjoin_mapjoin.q.out    |    1 +
 .../results/clientpositive/tez/mergejoin.q.out  |    3 +-
 .../test/results/clientpositive/tez/mrr.q.out   |    1 +
 .../test/results/clientpositive/tez/ptf.q.out   |    1 +
 .../clientpositive/tez/ptf_streaming.q.out      |    1 +
 .../clientpositive/tez/script_pipe.q.out        |    1 +
 .../clientpositive/tez/selectDistinctStar.q.out |    1 +
 .../clientpositive/tez/subquery_exists.q.out    |    1 +
 .../clientpositive/tez/subquery_in.q.out        |   16 +-
 .../results/clientpositive/tez/tez_dml.q.out    |    1 +
 .../tez/tez_dynpart_hashjoin_1.q.out            |    1 +
 .../tez/tez_dynpart_hashjoin_2.q.out            |    1 +
 .../tez/tez_dynpart_hashjoin_3.q.out            |    1 +
 .../clientpositive/tez/tez_join_tests.q.out     |    1 +
 .../clientpositive/tez/tez_joins_explain.q.out  |    1 +
 .../clientpositive/tez/tez_smb_main.q.out       |    1 +
 .../results/clientpositive/tez/tez_union.q.out  |    1 +
 .../results/clientpositive/tez/tez_union2.q.out |    1 +
 .../tez/tez_vector_dynpart_hashjoin_1.q.out     |    1 +
 .../tez/tez_vector_dynpart_hashjoin_2.q.out     |    1 +
 .../clientpositive/tez/transform_ppr1.q.out     |    1 +
 .../clientpositive/tez/transform_ppr2.q.out     |    1 +
 .../results/clientpositive/tez/union2.q.out     |    1 +
 .../results/clientpositive/tez/union3.q.out     |    1 +
 .../results/clientpositive/tez/union5.q.out     |    1 +
 .../results/clientpositive/tez/union7.q.out     |    1 +
 .../results/clientpositive/tez/union8.q.out     |    1 +
 .../results/clientpositive/tez/union9.q.out     |    1 +
 .../clientpositive/tez/vector_coalesce.q.out    |    1 +
 .../clientpositive/tez/vector_if_expr.q.out     |    1 +
 .../tez/vector_left_outer_join.q.out            |    1 +
 .../tez/vector_mapjoin_reduce.q.out             |    1 +
 .../clientpositive/tez/vectorization_0.q.out    |    1 +
 .../clientpositive/tez/vectorization_13.q.out   |    1 +
 .../clientpositive/tez/vectorization_14.q.out   |    1 +
 .../clientpositive/tez/vectorization_15.q.out   |    1 +
 .../clientpositive/tez/vectorization_16.q.out   |    1 +
 .../clientpositive/tez/vectorization_17.q.out   |    1 +
 .../clientpositive/tez/vectorization_7.q.out    |    1 +
 .../clientpositive/tez/vectorization_8.q.out    |    1 +
 .../clientpositive/tez/vectorization_9.q.out    |    1 +
 .../tez/vectorization_pushdown.q.out            |    1 +
 .../tez/vectorization_short_regress.q.out       |    5 +-
 .../clientpositive/tez/vectorized_mapjoin.q.out |    1 +
 .../tez/vectorized_nested_mapjoin.q.out         |    1 +
 .../tez/vectorized_shufflejoin.q.out            |    1 +
 .../vectorization_short_regress.q.out           |    4 +-
 182 files changed, 15861 insertions(+), 1136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f6be2180/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f6be2180/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --cc itests/src/test/resources/testconfiguration.properties
index 13b5113,68dbd0c..603159b
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@@ -494,9 -494,12 +494,13 @@@ minillap.shared.query.files=bucket_map_
    vectorized_dynamic_partition_pruning.q,\
    tez_multi_union.q,\
    tez_join.q,\
 -  tez_union_multiinsert.q
 +  tez_union_multiinsert.q,\
 +  udtf_get_splits.q
  
+ 
+ minillap.query.files=llap_udf.q
+ 
+ 
  encrypted.query.files=encryption_join_unencrypted_tbl.q,\
    encryption_insert_partition_static.q,\
    encryption_insert_partition_dynamic.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/f6be2180/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f6be2180/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f6be2180/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f6be2180/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f6be2180/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f6be2180/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
----------------------------------------------------------------------


[20/39] hive git commit: HIVE-13550: Get rid of wrapped LlapInputSplit/InputFormat classes

Posted by jd...@apache.org.
HIVE-13550: Get rid of wrapped LlapInputSplit/InputFormat classes


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0afaa8f6
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0afaa8f6
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0afaa8f6

Branch: refs/heads/master
Commit: 0afaa8f6dc60d51a01ba8085b1cb89624eafd3d0
Parents: 7b9096a
Author: Jason Dere <jd...@hortonworks.com>
Authored: Tue Apr 19 15:09:33 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Tue Apr 19 15:09:33 2016 -0700

----------------------------------------------------------------------
 .../hive/llap/ext/TestLlapInputSplit.java       |   26 +-
 .../hadoop/hive/llap/LlapInputFormat.java       |  392 ----
 llap-ext-client/pom.xml                         |   33 +
 .../hadoop/hive/llap/LlapBaseInputFormat.java   |  351 ++-
 .../hadoop/hive/llap/LlapRowInputFormat.java    |    4 +-
 .../apache/hive/llap/ext/LlapInputSplit.java    |   73 -
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |    1 -
 .../udf/generic/GenericUDTFExecuteSplits.java   |  124 -
 .../ql/udf/generic/GenericUDTFGetSplits.java    |   10 +-
 .../queries/clientpositive/udtf_get_splits.q    |   43 -
 .../clientpositive/llap/udtf_get_splits.q.out   | 2130 ------------------
 .../clientpositive/tez/udf_get_splits.q.out     |   73 -
 12 files changed, 389 insertions(+), 2871 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0afaa8f6/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
index 04da17e..8264190 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
@@ -10,6 +10,7 @@ import java.util.HashMap;
 
 import org.apache.hadoop.io.Text;
 
+import org.apache.hadoop.hive.llap.LlapInputSplit;
 import org.apache.hadoop.hive.llap.Schema;
 import org.apache.hadoop.hive.llap.FieldDesc;
 import org.apache.hadoop.hive.llap.TypeDesc;
@@ -40,7 +41,7 @@ public class TestLlapInputSplit {
     colDescs.add(new FieldDesc("col2", new TypeDesc(TypeDesc.Type.INT)));
     Schema schema = new Schema(colDescs);
 
-    org.apache.hadoop.hive.llap.LlapInputSplit split1 = new org.apache.hadoop.hive.llap.LlapInputSplit(
+    LlapInputSplit split1 = new LlapInputSplit(
         splitNum,
         planBytes,
         fragmentBytes,
@@ -52,35 +53,18 @@ public class TestLlapInputSplit {
     split1.write(dataOut);
     ByteArrayInputStream byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray());
     DataInputStream dataIn = new DataInputStream(byteInStream);
-    org.apache.hadoop.hive.llap.LlapInputSplit split2 = new org.apache.hadoop.hive.llap.LlapInputSplit();
+    LlapInputSplit split2 = new LlapInputSplit();
     split2.readFields(dataIn);
 
     // Did we read all the data?
     assertEquals(0, byteInStream.available());
 
     checkLlapSplits(split1, split2);
-
-    // Try JDBC LlapInputSplits
-    org.apache.hive.llap.ext.LlapInputSplit<Text> jdbcSplit1 =
-        new org.apache.hive.llap.ext.LlapInputSplit<Text>(split1, "org.apache.hadoop.hive.llap.LlapInputFormat");
-    byteOutStream.reset();
-    jdbcSplit1.write(dataOut);
-    byteInStream = new ByteArrayInputStream(byteOutStream.toByteArray());
-    dataIn = new DataInputStream(byteInStream);
-    org.apache.hive.llap.ext.LlapInputSplit<Text> jdbcSplit2 = new org.apache.hive.llap.ext.LlapInputSplit<Text>();
-    jdbcSplit2.readFields(dataIn);
-
-    assertEquals(0, byteInStream.available());
-
-    checkLlapSplits(
-        (org.apache.hadoop.hive.llap.LlapInputSplit) jdbcSplit1.getSplit(),
-        (org.apache.hadoop.hive.llap.LlapInputSplit) jdbcSplit2.getSplit());
-    assertEquals(jdbcSplit1.getInputFormat().getClass(), jdbcSplit2.getInputFormat().getClass());
   }
 
   static void checkLlapSplits(
-      org.apache.hadoop.hive.llap.LlapInputSplit split1,
-      org.apache.hadoop.hive.llap.LlapInputSplit split2) throws Exception {
+      LlapInputSplit split1,
+      LlapInputSplit split2) throws Exception {
 
     assertEquals(split1.getSplitNum(), split2.getSplitNum());
     assertArrayEquals(split1.getPlanBytes(), split2.getPlanBytes());

http://git-wip-us.apache.org/repos/asf/hive/blob/0afaa8f6/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
deleted file mode 100644
index 0930d60..0000000
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
+++ /dev/null
@@ -1,392 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.llap;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import com.google.common.collect.Lists;
-import com.google.protobuf.ByteString;
-
-import org.apache.commons.collections4.ListUtils;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.llap.LlapBaseRecordReader.ReaderEvent;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
-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.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.token.Token;
-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.runtime.api.events.TaskAttemptFailedEvent;
-import org.apache.tez.runtime.api.impl.TaskSpec;
-import org.apache.tez.runtime.api.impl.TezEvent;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.tez.dag.records.TezTaskAttemptID;
-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;
-
-
-public class LlapInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
-
-  private static final Logger LOG = LoggerFactory.getLogger(LlapInputFormat.class);
-
-  public LlapInputFormat() {
-  }
-
-  /*
-   * This proxy record reader has the duty of establishing a connected socket with LLAP, then fire
-   * off the work in the split to LLAP and finally return the connected socket back in an
-   * LlapRecordReader. The LlapRecordReader class reads the results from the socket.
-   */
-  @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 {
-    throw new IOException("These are not the splits you are looking for.");
-  }
-
-  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();
-    // 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());
-
-    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);
-
-
-    builder.setUsingTezAm(false);
-    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/0afaa8f6/llap-ext-client/pom.xml
----------------------------------------------------------------------
diff --git a/llap-ext-client/pom.xml b/llap-ext-client/pom.xml
index 5a7e385..fdf16cd 100644
--- a/llap-ext-client/pom.xml
+++ b/llap-ext-client/pom.xml
@@ -74,6 +74,39 @@
       <version>${hadoop.version}</version>
       <optional>true</optional>
     </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-api</artifactId>
+      <version>${tez.version}</version>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commmons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-runtime-internals</artifactId>
+      <version>${tez.version}</version>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commmons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
     <!-- test inter-project -->
     <dependency>
       <groupId>junit</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/0afaa8f6/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 61eb2ea..10d14c0 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
@@ -18,6 +18,8 @@ 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;
@@ -30,8 +32,28 @@ 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.SubmitWorkRequestProto;
+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;
@@ -44,16 +66,39 @@ 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.hive.llap.ext.LlapInputSplit;
+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",
@@ -82,8 +127,58 @@ public class LlapBaseInputFormat<V extends WritableComparable> implements InputF
 
   @Override
   public RecordReader<NullWritable, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+
     LlapInputSplit llapSplit = (LlapInputSplit) split;
-    return llapSplit.getInputFormat().getRecordReader(llapSplit.getSplit(), job, reporter);
+
+    // 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
@@ -112,10 +207,10 @@ public class LlapBaseInputFormat<V extends WritableComparable> implements InputF
       ResultSet res = stmt.executeQuery(sql);
       while (res.next()) {
         // deserialize split
-        DataInput in = new DataInputStream(res.getBinaryStream(3));
-        InputSplitWithLocationInfo is = (InputSplitWithLocationInfo)Class.forName(res.getString(2)).newInstance();
+        DataInput in = new DataInputStream(res.getBinaryStream(1));
+        InputSplitWithLocationInfo is = new LlapInputSplit();
         is.readFields(in);
-        ins.add(new LlapInputSplit(is, res.getString(1)));
+        ins.add(is);
       }
 
       res.close();
@@ -133,4 +228,250 @@ public class LlapBaseInputFormat<V extends WritableComparable> implements InputF
       // 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();
+    // 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());
+
+    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);
+
+
+    builder.setUsingTezAm(false);
+    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/0afaa8f6/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
index 6ecb0f9..56ad555 100644
--- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
+++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
@@ -3,6 +3,7 @@ package org.apache.hadoop.hive.llap;
 import java.io.IOException;
 
 import org.apache.hadoop.hive.llap.LlapBaseRecordReader;
+import org.apache.hadoop.hive.llap.LlapInputSplit;
 import org.apache.hadoop.hive.llap.LlapRowRecordReader;
 import org.apache.hadoop.hive.llap.Row;
 import org.apache.hadoop.hive.llap.Schema;
@@ -15,7 +16,6 @@ import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.hive.llap.ext.LlapInputSplit;
 
 
 public class LlapRowInputFormat implements InputFormat<NullWritable, Row> {
@@ -29,7 +29,7 @@ public class LlapRowInputFormat implements InputFormat<NullWritable, Row> {
   @Override
   public RecordReader<NullWritable, Row> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
       throws IOException {
-    LlapInputSplit<Text> llapSplit = (LlapInputSplit<Text>) split;
+    LlapInputSplit llapSplit = (LlapInputSplit) split;
     LlapBaseRecordReader<Text> reader = (LlapBaseRecordReader<Text>) baseInputFormat.getRecordReader(llapSplit, job, reporter);
     return new LlapRowRecordReader(job, reader.getSchema(), reader);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0afaa8f6/llap-ext-client/src/java/org/apache/hive/llap/ext/LlapInputSplit.java
----------------------------------------------------------------------
diff --git a/llap-ext-client/src/java/org/apache/hive/llap/ext/LlapInputSplit.java b/llap-ext-client/src/java/org/apache/hive/llap/ext/LlapInputSplit.java
deleted file mode 100644
index d8881c4..0000000
--- a/llap-ext-client/src/java/org/apache/hive/llap/ext/LlapInputSplit.java
+++ /dev/null
@@ -1,73 +0,0 @@
-package org.apache.hive.llap.ext;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
-import org.apache.hadoop.mapred.SplitLocationInfo;
-
-
-public class LlapInputSplit<V extends WritableComparable> implements InputSplitWithLocationInfo {
-  InputSplitWithLocationInfo nativeSplit;
-  String inputFormatClassName;
-
-  public LlapInputSplit() {
-  }
-
-  public LlapInputSplit(InputSplitWithLocationInfo nativeSplit, String inputFormatClassName) {
-    this.nativeSplit = nativeSplit;
-    this.inputFormatClassName = inputFormatClassName;
-  }
-
-  @Override
-  public long getLength() throws IOException {
-    return nativeSplit.getLength();
-  }
-
-  @Override
-  public String[] getLocations() throws IOException {
-    return nativeSplit.getLocations();
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeUTF(inputFormatClassName);
-    out.writeUTF(nativeSplit.getClass().getName());
-    nativeSplit.write(out);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    inputFormatClassName = in.readUTF();
-    String splitClass = in.readUTF();
-    try {
-      nativeSplit = (InputSplitWithLocationInfo)Class.forName(splitClass).newInstance();
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-    nativeSplit.readFields(in);
-  }
-
-  @Override
-  public SplitLocationInfo[] getLocationInfo() throws IOException {
-    return nativeSplit.getLocationInfo();
-  }
-
-  public InputSplit getSplit() {
-    return nativeSplit;
-  }
-
-  public InputFormat<NullWritable, V> getInputFormat() throws IOException {
-    try {
-      return (InputFormat<NullWritable, V>) Class.forName(inputFormatClassName)
-          .newInstance();
-    } catch(Exception e) {
-      throw new IOException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/0afaa8f6/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 54bd830..6b25ce1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -447,7 +447,6 @@ public final class FunctionRegistry {
     system.registerGenericUDTF("posexplode", GenericUDTFPosExplode.class);
     system.registerGenericUDTF("stack", GenericUDTFStack.class);
     system.registerGenericUDTF("get_splits", GenericUDTFGetSplits.class);
-    system.registerGenericUDTF("execute_splits", GenericUDTFExecuteSplits.class);
 
     //PTF declarations
     system.registerGenericUDF(LEAD_FUNC_NAME, GenericUDFLead.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/0afaa8f6/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFExecuteSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFExecuteSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFExecuteSplits.java
deleted file mode 100644
index 12759ab..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFExecuteSplits.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.udf.generic;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.hadoop.hive.ql.exec.Description;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.ql.udf.UDFType;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDTFGetSplits.PlanFragment;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.RecordReader;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * GenericUDTFExecuteSplits.
- *
- */
-@Description(name = "execute_splits", value = "_FUNC_(string,int) - "
-    + "Returns an array of length int serialized splits for the referenced tables string.")
-@UDFType(deterministic = false)
-public class GenericUDTFExecuteSplits extends GenericUDTFGetSplits {
-
-  private static final Logger LOG = LoggerFactory.getLogger(GenericUDTFExecuteSplits.class);
-
-  @Override
-  public StructObjectInspector initialize(ObjectInspector[] arguments)
-    throws UDFArgumentException {
-
-    LOG.debug("initializing ExecuteSplits");
-
-    if (SessionState.get() == null || SessionState.get().getConf() == null) {
-      throw new IllegalStateException("Cannot run execute splits outside HS2");
-    }
-
-    if (arguments.length != 2) {
-      throw new UDFArgumentLengthException("The function execute_splits accepts 2 arguments.");
-    } else if (!(arguments[0] instanceof StringObjectInspector)) {
-      LOG.error("Got "+arguments[0].getTypeName()+" instead of string.");
-      throw new UDFArgumentTypeException(0, "\""
-          + "string\" is expected at function execute_splits, " + "but \""
-          + arguments[0].getTypeName() + "\" is found");
-    } else if (!(arguments[1] instanceof IntObjectInspector)) {
-      LOG.error("Got "+arguments[1].getTypeName()+" instead of int.");
-      throw new UDFArgumentTypeException(1, "\""
-          + "int\" is expected at function execute_splits, " + "but \""
-          + arguments[1].getTypeName() + "\" is found");
-    }
-
-    stringOI = (StringObjectInspector) arguments[0];
-    intOI = (IntObjectInspector) arguments[1];
-
-    List<String> names = Arrays.asList("split_num","value");
-    List<ObjectInspector> fieldOIs = Arrays.<ObjectInspector>asList(
-      PrimitiveObjectInspectorFactory.javaIntObjectInspector,
-      PrimitiveObjectInspectorFactory.javaStringObjectInspector);
-    StructObjectInspector outputOI = ObjectInspectorFactory.getStandardStructObjectInspector(names, fieldOIs);
-
-    LOG.debug("done initializing GenericUDTFExecuteSplits");
-    return outputOI;
-  }
-
-  @Override
-  public void process(Object[] arguments) throws HiveException {
-
-    String query = stringOI.getPrimitiveJavaObject(arguments[0]);
-    int num = intOI.get(arguments[1]);
-
-    PlanFragment fragment = createPlanFragment(query, num);
-    try {
-      InputFormat<NullWritable, Text> format = (InputFormat<NullWritable,Text>)(Class.forName("org.apache.hadoop.hive.llap.LlapInputFormat").newInstance());
-      int index = 0;
-      for (InputSplit s: getSplits(jc, num, fragment.work, fragment.schema)) {
-        RecordReader<NullWritable, Text> reader = format.getRecordReader(s,fragment.jc,null);
-        Text value = reader.createValue();
-        NullWritable key = reader.createKey();
-        index++;
-        while(reader.next(key,value)) {
-          Object[] os = new Object[2];
-          os[0] = index;
-          os[1] = value.toString();
-          forward(os);
-        }
-      }
-    } catch(Exception e) {
-      throw new HiveException(e);
-    }
-  }
-
-  @Override
-  public void close() throws HiveException {
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/0afaa8f6/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
index 51027a7..9a52c7d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
@@ -150,10 +150,8 @@ public class GenericUDTFGetSplits extends GenericUDTF {
     stringOI = (StringObjectInspector) arguments[0];
     intOI = (IntObjectInspector) arguments[1];
 
-    List<String> names = Arrays.asList("if_class","split_class","split");
+    List<String> names = Arrays.asList("split");
     List<ObjectInspector> fieldOIs = Arrays.<ObjectInspector>asList(
-      PrimitiveObjectInspectorFactory.javaStringObjectInspector,
-      PrimitiveObjectInspectorFactory.javaStringObjectInspector,
       PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector);
     StructObjectInspector outputOI = ObjectInspectorFactory.getStandardStructObjectInspector(names, fieldOIs);
 
@@ -185,13 +183,11 @@ public class GenericUDTFGetSplits extends GenericUDTF {
 
     try {
       for (InputSplit s: getSplits(jc, num, tezWork, schema)) {
-        Object[] os = new Object[3];
-        os[0] = LLAP_INTERNAL_INPUT_FORMAT_NAME;
-        os[1] = s.getClass().getName();
+        Object[] os = new Object[1];
         bos.reset();
         s.write(dos);
         byte[] frozen = bos.toByteArray();
-        os[2] = frozen;
+        os[0] = frozen;
         forward(os);
       }
     } catch(Exception e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0afaa8f6/ql/src/test/queries/clientpositive/udtf_get_splits.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udtf_get_splits.q b/ql/src/test/queries/clientpositive/udtf_get_splits.q
deleted file mode 100644
index f378dca..0000000
--- a/ql/src/test/queries/clientpositive/udtf_get_splits.q
+++ /dev/null
@@ -1,43 +0,0 @@
-set hive.fetch.task.conversion=more;
-set hive.mapred.mode=nonstrict;
-set mapred.max.split.size=100;
-set mapred.min.split.size.per.node=100;
-set mapred.min.split.size.per.rack=100;
-set mapred.max.split.size=100;
-set tez.grouping.max-size=100;
-set tez.grouping.min-size=100;
-
-DESCRIBE FUNCTION get_splits;
-DESCRIBE FUNCTION execute_splits;
-
-select r1, r2, floor(length(r3)/100000)
-from
-  (select
-    get_splits(
-      "select key, count(*) from srcpart where key % 2 = 0 group by key",
-      5) as (r1, r2, r3)) t;
-
-select r1, r2, floor(length(r3)/100000)
-from
-  (select
-    get_splits(
-      "select key from srcpart where key % 2 = 0",
-      5) as (r1, r2, r3)) t;
-
-show tables;
-
-select r1, r2
-from
-  (select
-    execute_splits(
-      "select key from srcpart where key % 2 = 0",
-      1) as (r1, r2)) t;
-
-select r1, r2
-from
-  (select
-    execute_splits(
-      "select key from srcpart where key % 2 = 0",
-      5) as (r1, r2)) t;
-
-select count(*) from (select key from srcpart where key % 2 = 0) t;

http://git-wip-us.apache.org/repos/asf/hive/blob/0afaa8f6/ql/src/test/results/clientpositive/llap/udtf_get_splits.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/udtf_get_splits.q.out b/ql/src/test/results/clientpositive/llap/udtf_get_splits.q.out
deleted file mode 100644
index 2f17a91..0000000
--- a/ql/src/test/results/clientpositive/llap/udtf_get_splits.q.out
+++ /dev/null
@@ -1,2130 +0,0 @@
-PREHOOK: query: DESCRIBE FUNCTION get_splits
-PREHOOK: type: DESCFUNCTION
-POSTHOOK: query: DESCRIBE FUNCTION get_splits
-POSTHOOK: type: DESCFUNCTION
-get_splits(string,int) - Returns an array of length int serialized splits for the referenced tables string.
-PREHOOK: query: DESCRIBE FUNCTION execute_splits
-PREHOOK: type: DESCFUNCTION
-POSTHOOK: query: DESCRIBE FUNCTION execute_splits
-POSTHOOK: type: DESCFUNCTION
-execute_splits(string,int) - Returns an array of length int serialized splits for the referenced tables string.
-PREHOOK: query: select r1, r2, floor(length(r3)/100000)
-from
-  (select
-    get_splits(
-      "select key, count(*) from srcpart where key % 2 = 0 group by key",
-      5) as (r1, r2, r3)) t
-PREHOOK: type: QUERY
-PREHOOK: Input: _dummy_database@_dummy_table
-#### A masked pattern was here ####
-POSTHOOK: query: select r1, r2, floor(length(r3)/100000)
-from
-  (select
-    get_splits(
-      "select key, count(*) from srcpart where key % 2 = 0 group by key",
-      5) as (r1, r2, r3)) t
-POSTHOOK: type: QUERY
-POSTHOOK: Input: _dummy_database@_dummy_table
-#### A masked pattern was here ####
-PREHOOK: query: select r1, r2, floor(length(r3)/100000)
-from
-  (select
-    get_splits(
-      "select key, count(*) from srcpart where key % 2 = 0 group by key",
-      5) as (r1, r2, r3)) t
-PREHOOK: type: QUERY
-PREHOOK: Input: default@srcpart
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
-PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
-PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
-PREHOOK: Output: database:default
-PREHOOK: Output: default@#### A masked pattern was here ####
-POSTHOOK: query: select r1, r2, floor(length(r3)/100000)
-from
-  (select
-    get_splits(
-      "select key, count(*) from srcpart where key % 2 = 0 group by key",
-      5) as (r1, r2, r3)) t
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@srcpart
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
-POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
-POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@#### A masked pattern was here ####
-org.apache.hadoop.hive.llap.LlapInputFormat	org.apache.hadoop.hive.llap.LlapInputSplit	1
-PREHOOK: query: select r1, r2, floor(length(r3)/100000)
-from
-  (select
-    get_splits(
-      "select key from srcpart where key % 2 = 0",
-      5) as (r1, r2, r3)) t
-PREHOOK: type: QUERY
-PREHOOK: Input: _dummy_database@_dummy_table
-#### A masked pattern was here ####
-POSTHOOK: query: select r1, r2, floor(length(r3)/100000)
-from
-  (select
-    get_splits(
-      "select key from srcpart where key % 2 = 0",
-      5) as (r1, r2, r3)) t
-POSTHOOK: type: QUERY
-POSTHOOK: Input: _dummy_database@_dummy_table
-#### A masked pattern was here ####
-org.apache.hadoop.hive.llap.LlapInputFormat	org.apache.hadoop.hive.llap.LlapInputSplit	1
-org.apache.hadoop.hive.llap.LlapInputFormat	org.apache.hadoop.hive.llap.LlapInputSplit	1
-org.apache.hadoop.hive.llap.LlapInputFormat	org.apache.hadoop.hive.llap.LlapInputSplit	1
-org.apache.hadoop.hive.llap.LlapInputFormat	org.apache.hadoop.hive.llap.LlapInputSplit	1
-PREHOOK: query: show tables
-PREHOOK: type: SHOWTABLES
-PREHOOK: Input: database:default
-POSTHOOK: query: show tables
-POSTHOOK: type: SHOWTABLES
-POSTHOOK: Input: database:default
-alltypesorc
-cbo_t1
-cbo_t2
-cbo_t3
-lineitem
-part
-src
-src1
-src_cbo
-src_json
-src_sequencefile
-src_thrift
-srcbucket
-srcbucket2
-srcpart
-#### A masked pattern was here ####
-PREHOOK: query: select r1, r2
-from
-  (select
-    execute_splits(
-      "select key from srcpart where key % 2 = 0",
-      1) as (r1, r2)) t
-PREHOOK: type: QUERY
-PREHOOK: Input: _dummy_database@_dummy_table
-#### A masked pattern was here ####
-POSTHOOK: query: select r1, r2
-from
-  (select
-    execute_splits(
-      "select key from srcpart where key % 2 = 0",
-      1) as (r1, r2)) t
-POSTHOOK: type: QUERY
-POSTHOOK: Input: _dummy_database@_dummy_table
-#### A masked pattern was here ####
-1	238
-1	86
-1	278
-1	98
-1	484
-1	150
-1	224
-1	66
-1	128
-1	146
-1	406
-1	374
-1	152
-1	82
-1	166
-1	430
-1	252
-1	292
-1	338
-1	446
-1	394
-1	482
-1	174
-1	494
-1	466
-1	208
-1	174
-1	396
-1	162
-1	266
-1	342
-1	0
-1	128
-1	316
-1	302
-1	438
-1	170
-1	20
-1	378
-1	92
-1	72
-1	4
-1	280
-1	208
-1	356
-1	382
-1	498
-1	386
-1	192
-1	286
-1	176
-1	54
-1	138
-1	216
-1	430
-1	278
-1	176
-1	318
-1	332
-1	180
-1	284
-1	12
-1	230
-1	260
-1	404
-1	384
-1	272
-1	138
-1	84
-1	348
-1	466
-1	58
-1	8
-1	230
-1	208
-1	348
-1	24
-1	172
-1	42
-1	158
-1	496
-1	0
-1	322
-1	468
-1	454
-1	100
-1	298
-1	418
-1	96
-1	26
-1	230
-1	120
-1	404
-1	436
-1	156
-1	468
-1	308
-1	196
-1	288
-1	98
-1	282
-1	318
-1	318
-1	470
-1	316
-1	0
-1	490
-1	364
-1	118
-1	134
-1	282
-1	138
-1	238
-1	118
-1	72
-1	90
-1	10
-1	306
-1	224
-1	242
-1	392
-1	272
-1	242
-1	452
-1	226
-1	402
-1	396
-1	58
-1	336
-1	168
-1	34
-1	472
-1	322
-1	498
-1	160
-1	42
-1	430
-1	458
-1	78
-1	76
-1	492
-1	218
-1	228
-1	138
-1	30
-1	64
-1	468
-1	76
-1	74
-1	342
-1	230
-1	368
-1	296
-1	216
-1	344
-1	274
-1	116
-1	256
-1	70
-1	480
-1	288
-1	244
-1	438
-1	128
-1	432
-1	202
-1	316
-1	280
-1	2
-1	80
-1	44
-1	104
-1	466
-1	366
-1	406
-1	190
-1	406
-1	114
-1	258
-1	90
-1	262
-1	348
-1	424
-1	12
-1	396
-1	164
-1	454
-1	478
-1	298
-1	164
-1	424
-1	382
-1	70
-1	480
-1	24
-1	104
-1	70
-1	438
-1	414
-1	200
-1	360
-1	248
-1	444
-1	120
-1	230
-1	478
-1	178
-1	468
-1	310
-1	460
-1	480
-1	136
-1	172
-1	214
-1	462
-1	406
-1	454
-1	384
-1	256
-1	26
-1	134
-1	384
-1	18
-1	462
-1	492
-1	100
-1	298
-1	498
-1	146
-1	458
-1	362
-1	186
-1	348
-1	18
-1	344
-1	84
-1	28
-1	448
-1	152
-1	348
-1	194
-1	414
-1	222
-1	126
-1	90
-1	400
-1	200
-2	238
-2	86
-2	278
-2	98
-2	484
-2	150
-2	224
-2	66
-2	128
-2	146
-2	406
-2	374
-2	152
-2	82
-2	166
-2	430
-2	252
-2	292
-2	338
-2	446
-2	394
-2	482
-2	174
-2	494
-2	466
-2	208
-2	174
-2	396
-2	162
-2	266
-2	342
-2	0
-2	128
-2	316
-2	302
-2	438
-2	170
-2	20
-2	378
-2	92
-2	72
-2	4
-2	280
-2	208
-2	356
-2	382
-2	498
-2	386
-2	192
-2	286
-2	176
-2	54
-2	138
-2	216
-2	430
-2	278
-2	176
-2	318
-2	332
-2	180
-2	284
-2	12
-2	230
-2	260
-2	404
-2	384
-2	272
-2	138
-2	84
-2	348
-2	466
-2	58
-2	8
-2	230
-2	208
-2	348
-2	24
-2	172
-2	42
-2	158
-2	496
-2	0
-2	322
-2	468
-2	454
-2	100
-2	298
-2	418
-2	96
-2	26
-2	230
-2	120
-2	404
-2	436
-2	156
-2	468
-2	308
-2	196
-2	288
-2	98
-2	282
-2	318
-2	318
-2	470
-2	316
-2	0
-2	490
-2	364
-2	118
-2	134
-2	282
-2	138
-2	238
-2	118
-2	72
-2	90
-2	10
-2	306
-2	224
-2	242
-2	392
-2	272
-2	242
-2	452
-2	226
-2	402
-2	396
-2	58
-2	336
-2	168
-2	34
-2	472
-2	322
-2	498
-2	160
-2	42
-2	430
-2	458
-2	78
-2	76
-2	492
-2	218
-2	228
-2	138
-2	30
-2	64
-2	468
-2	76
-2	74
-2	342
-2	230
-2	368
-2	296
-2	216
-2	344
-2	274
-2	116
-2	256
-2	70
-2	480
-2	288
-2	244
-2	438
-2	128
-2	432
-2	202
-2	316
-2	280
-2	2
-2	80
-2	44
-2	104
-2	466
-2	366
-2	406
-2	190
-2	406
-2	114
-2	258
-2	90
-2	262
-2	348
-2	424
-2	12
-2	396
-2	164
-2	454
-2	478
-2	298
-2	164
-2	424
-2	382
-2	70
-2	480
-2	24
-2	104
-2	70
-2	438
-2	414
-2	200
-2	360
-2	248
-2	444
-2	120
-2	230
-2	478
-2	178
-2	468
-2	310
-2	460
-2	480
-2	136
-2	172
-2	214
-2	462
-2	406
-2	454
-2	384
-2	256
-2	26
-2	134
-2	384
-2	18
-2	462
-2	492
-2	100
-2	298
-2	498
-2	146
-2	458
-2	362
-2	186
-2	348
-2	18
-2	344
-2	84
-2	28
-2	448
-2	152
-2	348
-2	194
-2	414
-2	222
-2	126
-2	90
-2	400
-2	200
-3	238
-3	86
-3	278
-3	98
-3	484
-3	150
-3	224
-3	66
-3	128
-3	146
-3	406
-3	374
-3	152
-3	82
-3	166
-3	430
-3	252
-3	292
-3	338
-3	446
-3	394
-3	482
-3	174
-3	494
-3	466
-3	208
-3	174
-3	396
-3	162
-3	266
-3	342
-3	0
-3	128
-3	316
-3	302
-3	438
-3	170
-3	20
-3	378
-3	92
-3	72
-3	4
-3	280
-3	208
-3	356
-3	382
-3	498
-3	386
-3	192
-3	286
-3	176
-3	54
-3	138
-3	216
-3	430
-3	278
-3	176
-3	318
-3	332
-3	180
-3	284
-3	12
-3	230
-3	260
-3	404
-3	384
-3	272
-3	138
-3	84
-3	348
-3	466
-3	58
-3	8
-3	230
-3	208
-3	348
-3	24
-3	172
-3	42
-3	158
-3	496
-3	0
-3	322
-3	468
-3	454
-3	100
-3	298
-3	418
-3	96
-3	26
-3	230
-3	120
-3	404
-3	436
-3	156
-3	468
-3	308
-3	196
-3	288
-3	98
-3	282
-3	318
-3	318
-3	470
-3	316
-3	0
-3	490
-3	364
-3	118
-3	134
-3	282
-3	138
-3	238
-3	118
-3	72
-3	90
-3	10
-3	306
-3	224
-3	242
-3	392
-3	272
-3	242
-3	452
-3	226
-3	402
-3	396
-3	58
-3	336
-3	168
-3	34
-3	472
-3	322
-3	498
-3	160
-3	42
-3	430
-3	458
-3	78
-3	76
-3	492
-3	218
-3	228
-3	138
-3	30
-3	64
-3	468
-3	76
-3	74
-3	342
-3	230
-3	368
-3	296
-3	216
-3	344
-3	274
-3	116
-3	256
-3	70
-3	480
-3	288
-3	244
-3	438
-3	128
-3	432
-3	202
-3	316
-3	280
-3	2
-3	80
-3	44
-3	104
-3	466
-3	366
-3	406
-3	190
-3	406
-3	114
-3	258
-3	90
-3	262
-3	348
-3	424
-3	12
-3	396
-3	164
-3	454
-3	478
-3	298
-3	164
-3	424
-3	382
-3	70
-3	480
-3	24
-3	104
-3	70
-3	438
-3	414
-3	200
-3	360
-3	248
-3	444
-3	120
-3	230
-3	478
-3	178
-3	468
-3	310
-3	460
-3	480
-3	136
-3	172
-3	214
-3	462
-3	406
-3	454
-3	384
-3	256
-3	26
-3	134
-3	384
-3	18
-3	462
-3	492
-3	100
-3	298
-3	498
-3	146
-3	458
-3	362
-3	186
-3	348
-3	18
-3	344
-3	84
-3	28
-3	448
-3	152
-3	348
-3	194
-3	414
-3	222
-3	126
-3	90
-3	400
-3	200
-4	238
-4	86
-4	278
-4	98
-4	484
-4	150
-4	224
-4	66
-4	128
-4	146
-4	406
-4	374
-4	152
-4	82
-4	166
-4	430
-4	252
-4	292
-4	338
-4	446
-4	394
-4	482
-4	174
-4	494
-4	466
-4	208
-4	174
-4	396
-4	162
-4	266
-4	342
-4	0
-4	128
-4	316
-4	302
-4	438
-4	170
-4	20
-4	378
-4	92
-4	72
-4	4
-4	280
-4	208
-4	356
-4	382
-4	498
-4	386
-4	192
-4	286
-4	176
-4	54
-4	138
-4	216
-4	430
-4	278
-4	176
-4	318
-4	332
-4	180
-4	284
-4	12
-4	230
-4	260
-4	404
-4	384
-4	272
-4	138
-4	84
-4	348
-4	466
-4	58
-4	8
-4	230
-4	208
-4	348
-4	24
-4	172
-4	42
-4	158
-4	496
-4	0
-4	322
-4	468
-4	454
-4	100
-4	298
-4	418
-4	96
-4	26
-4	230
-4	120
-4	404
-4	436
-4	156
-4	468
-4	308
-4	196
-4	288
-4	98
-4	282
-4	318
-4	318
-4	470
-4	316
-4	0
-4	490
-4	364
-4	118
-4	134
-4	282
-4	138
-4	238
-4	118
-4	72
-4	90
-4	10
-4	306
-4	224
-4	242
-4	392
-4	272
-4	242
-4	452
-4	226
-4	402
-4	396
-4	58
-4	336
-4	168
-4	34
-4	472
-4	322
-4	498
-4	160
-4	42
-4	430
-4	458
-4	78
-4	76
-4	492
-4	218
-4	228
-4	138
-4	30
-4	64
-4	468
-4	76
-4	74
-4	342
-4	230
-4	368
-4	296
-4	216
-4	344
-4	274
-4	116
-4	256
-4	70
-4	480
-4	288
-4	244
-4	438
-4	128
-4	432
-4	202
-4	316
-4	280
-4	2
-4	80
-4	44
-4	104
-4	466
-4	366
-4	406
-4	190
-4	406
-4	114
-4	258
-4	90
-4	262
-4	348
-4	424
-4	12
-4	396
-4	164
-4	454
-4	478
-4	298
-4	164
-4	424
-4	382
-4	70
-4	480
-4	24
-4	104
-4	70
-4	438
-4	414
-4	200
-4	360
-4	248
-4	444
-4	120
-4	230
-4	478
-4	178
-4	468
-4	310
-4	460
-4	480
-4	136
-4	172
-4	214
-4	462
-4	406
-4	454
-4	384
-4	256
-4	26
-4	134
-4	384
-4	18
-4	462
-4	492
-4	100
-4	298
-4	498
-4	146
-4	458
-4	362
-4	186
-4	348
-4	18
-4	344
-4	84
-4	28
-4	448
-4	152
-4	348
-4	194
-4	414
-4	222
-4	126
-4	90
-4	400
-4	200
-PREHOOK: query: select r1, r2
-from
-  (select
-    execute_splits(
-      "select key from srcpart where key % 2 = 0",
-      5) as (r1, r2)) t
-PREHOOK: type: QUERY
-PREHOOK: Input: _dummy_database@_dummy_table
-#### A masked pattern was here ####
-POSTHOOK: query: select r1, r2
-from
-  (select
-    execute_splits(
-      "select key from srcpart where key % 2 = 0",
-      5) as (r1, r2)) t
-POSTHOOK: type: QUERY
-POSTHOOK: Input: _dummy_database@_dummy_table
-#### A masked pattern was here ####
-1	238
-1	86
-1	278
-1	98
-1	484
-1	150
-1	224
-1	66
-1	128
-1	146
-1	406
-1	374
-1	152
-1	82
-1	166
-1	430
-1	252
-1	292
-1	338
-1	446
-1	394
-1	482
-1	174
-1	494
-1	466
-1	208
-1	174
-1	396
-1	162
-1	266
-1	342
-1	0
-1	128
-1	316
-1	302
-1	438
-1	170
-1	20
-1	378
-1	92
-1	72
-1	4
-1	280
-1	208
-1	356
-1	382
-1	498
-1	386
-1	192
-1	286
-1	176
-1	54
-1	138
-1	216
-1	430
-1	278
-1	176
-1	318
-1	332
-1	180
-1	284
-1	12
-1	230
-1	260
-1	404
-1	384
-1	272
-1	138
-1	84
-1	348
-1	466
-1	58
-1	8
-1	230
-1	208
-1	348
-1	24
-1	172
-1	42
-1	158
-1	496
-1	0
-1	322
-1	468
-1	454
-1	100
-1	298
-1	418
-1	96
-1	26
-1	230
-1	120
-1	404
-1	436
-1	156
-1	468
-1	308
-1	196
-1	288
-1	98
-1	282
-1	318
-1	318
-1	470
-1	316
-1	0
-1	490
-1	364
-1	118
-1	134
-1	282
-1	138
-1	238
-1	118
-1	72
-1	90
-1	10
-1	306
-1	224
-1	242
-1	392
-1	272
-1	242
-1	452
-1	226
-1	402
-1	396
-1	58
-1	336
-1	168
-1	34
-1	472
-1	322
-1	498
-1	160
-1	42
-1	430
-1	458
-1	78
-1	76
-1	492
-1	218
-1	228
-1	138
-1	30
-1	64
-1	468
-1	76
-1	74
-1	342
-1	230
-1	368
-1	296
-1	216
-1	344
-1	274
-1	116
-1	256
-1	70
-1	480
-1	288
-1	244
-1	438
-1	128
-1	432
-1	202
-1	316
-1	280
-1	2
-1	80
-1	44
-1	104
-1	466
-1	366
-1	406
-1	190
-1	406
-1	114
-1	258
-1	90
-1	262
-1	348
-1	424
-1	12
-1	396
-1	164
-1	454
-1	478
-1	298
-1	164
-1	424
-1	382
-1	70
-1	480
-1	24
-1	104
-1	70
-1	438
-1	414
-1	200
-1	360
-1	248
-1	444
-1	120
-1	230
-1	478
-1	178
-1	468
-1	310
-1	460
-1	480
-1	136
-1	172
-1	214
-1	462
-1	406
-1	454
-1	384
-1	256
-1	26
-1	134
-1	384
-1	18
-1	462
-1	492
-1	100
-1	298
-1	498
-1	146
-1	458
-1	362
-1	186
-1	348
-1	18
-1	344
-1	84
-1	28
-1	448
-1	152
-1	348
-1	194
-1	414
-1	222
-1	126
-1	90
-1	400
-1	200
-2	238
-2	86
-2	278
-2	98
-2	484
-2	150
-2	224
-2	66
-2	128
-2	146
-2	406
-2	374
-2	152
-2	82
-2	166
-2	430
-2	252
-2	292
-2	338
-2	446
-2	394
-2	482
-2	174
-2	494
-2	466
-2	208
-2	174
-2	396
-2	162
-2	266
-2	342
-2	0
-2	128
-2	316
-2	302
-2	438
-2	170
-2	20
-2	378
-2	92
-2	72
-2	4
-2	280
-2	208
-2	356
-2	382
-2	498
-2	386
-2	192
-2	286
-2	176
-2	54
-2	138
-2	216
-2	430
-2	278
-2	176
-2	318
-2	332
-2	180
-2	284
-2	12
-2	230
-2	260
-2	404
-2	384
-2	272
-2	138
-2	84
-2	348
-2	466
-2	58
-2	8
-2	230
-2	208
-2	348
-2	24
-2	172
-2	42
-2	158
-2	496
-2	0
-2	322
-2	468
-2	454
-2	100
-2	298
-2	418
-2	96
-2	26
-2	230
-2	120
-2	404
-2	436
-2	156
-2	468
-2	308
-2	196
-2	288
-2	98
-2	282
-2	318
-2	318
-2	470
-2	316
-2	0
-2	490
-2	364
-2	118
-2	134
-2	282
-2	138
-2	238
-2	118
-2	72
-2	90
-2	10
-2	306
-2	224
-2	242
-2	392
-2	272
-2	242
-2	452
-2	226
-2	402
-2	396
-2	58
-2	336
-2	168
-2	34
-2	472
-2	322
-2	498
-2	160
-2	42
-2	430
-2	458
-2	78
-2	76
-2	492
-2	218
-2	228
-2	138
-2	30
-2	64
-2	468
-2	76
-2	74
-2	342
-2	230
-2	368
-2	296
-2	216
-2	344
-2	274
-2	116
-2	256
-2	70
-2	480
-2	288
-2	244
-2	438
-2	128
-2	432
-2	202
-2	316
-2	280
-2	2
-2	80
-2	44
-2	104
-2	466
-2	366
-2	406
-2	190
-2	406
-2	114
-2	258
-2	90
-2	262
-2	348
-2	424
-2	12
-2	396
-2	164
-2	454
-2	478
-2	298
-2	164
-2	424
-2	382
-2	70
-2	480
-2	24
-2	104
-2	70
-2	438
-2	414
-2	200
-2	360
-2	248
-2	444
-2	120
-2	230
-2	478
-2	178
-2	468
-2	310
-2	460
-2	480
-2	136
-2	172
-2	214
-2	462
-2	406
-2	454
-2	384
-2	256
-2	26
-2	134
-2	384
-2	18
-2	462
-2	492
-2	100
-2	298
-2	498
-2	146
-2	458
-2	362
-2	186
-2	348
-2	18
-2	344
-2	84
-2	28
-2	448
-2	152
-2	348
-2	194
-2	414
-2	222
-2	126
-2	90
-2	400
-2	200
-3	238
-3	86
-3	278
-3	98
-3	484
-3	150
-3	224
-3	66
-3	128
-3	146
-3	406
-3	374
-3	152
-3	82
-3	166
-3	430
-3	252
-3	292
-3	338
-3	446
-3	394
-3	482
-3	174
-3	494
-3	466
-3	208
-3	174
-3	396
-3	162
-3	266
-3	342
-3	0
-3	128
-3	316
-3	302
-3	438
-3	170
-3	20
-3	378
-3	92
-3	72
-3	4
-3	280
-3	208
-3	356
-3	382
-3	498
-3	386
-3	192
-3	286
-3	176
-3	54
-3	138
-3	216
-3	430
-3	278
-3	176
-3	318
-3	332
-3	180
-3	284
-3	12
-3	230
-3	260
-3	404
-3	384
-3	272
-3	138
-3	84
-3	348
-3	466
-3	58
-3	8
-3	230
-3	208
-3	348
-3	24
-3	172
-3	42
-3	158
-3	496
-3	0
-3	322
-3	468
-3	454
-3	100
-3	298
-3	418
-3	96
-3	26
-3	230
-3	120
-3	404
-3	436
-3	156
-3	468
-3	308
-3	196
-3	288
-3	98
-3	282
-3	318
-3	318
-3	470
-3	316
-3	0
-3	490
-3	364
-3	118
-3	134
-3	282
-3	138
-3	238
-3	118
-3	72
-3	90
-3	10
-3	306
-3	224
-3	242
-3	392
-3	272
-3	242
-3	452
-3	226
-3	402
-3	396
-3	58
-3	336
-3	168
-3	34
-3	472
-3	322
-3	498
-3	160
-3	42
-3	430
-3	458
-3	78
-3	76
-3	492
-3	218
-3	228
-3	138
-3	30
-3	64
-3	468
-3	76
-3	74
-3	342
-3	230
-3	368
-3	296
-3	216
-3	344
-3	274
-3	116
-3	256
-3	70
-3	480
-3	288
-3	244
-3	438
-3	128
-3	432
-3	202
-3	316
-3	280
-3	2
-3	80
-3	44
-3	104
-3	466
-3	366
-3	406
-3	190
-3	406
-3	114
-3	258
-3	90
-3	262
-3	348
-3	424
-3	12
-3	396
-3	164
-3	454
-3	478
-3	298
-3	164
-3	424
-3	382
-3	70
-3	480
-3	24
-3	104
-3	70
-3	438
-3	414
-3	200
-3	360
-3	248
-3	444
-3	120
-3	230
-3	478
-3	178
-3	468
-3	310
-3	460
-3	480
-3	136
-3	172
-3	214
-3	462
-3	406
-3	454
-3	384
-3	256
-3	26
-3	134
-3	384
-3	18
-3	462
-3	492
-3	100
-3	298
-3	498
-3	146
-3	458
-3	362
-3	186
-3	348
-3	18
-3	344
-3	84
-3	28
-3	448
-3	152
-3	348
-3	194
-3	414
-3	222
-3	126
-3	90
-3	400
-3	200
-4	238
-4	86
-4	278
-4	98
-4	484
-4	150
-4	224
-4	66
-4	128
-4	146
-4	406
-4	374
-4	152
-4	82
-4	166
-4	430
-4	252
-4	292
-4	338
-4	446
-4	394
-4	482
-4	174
-4	494
-4	466
-4	208
-4	174
-4	396
-4	162
-4	266
-4	342
-4	0
-4	128
-4	316
-4	302
-4	438
-4	170
-4	20
-4	378
-4	92
-4	72
-4	4
-4	280
-4	208
-4	356
-4	382
-4	498
-4	386
-4	192
-4	286
-4	176
-4	54
-4	138
-4	216
-4	430
-4	278
-4	176
-4	318
-4	332
-4	180
-4	284
-4	12
-4	230
-4	260
-4	404
-4	384
-4	272
-4	138
-4	84
-4	348
-4	466
-4	58
-4	8
-4	230
-4	208
-4	348
-4	24
-4	172
-4	42
-4	158
-4	496
-4	0
-4	322
-4	468
-4	454
-4	100
-4	298
-4	418
-4	96
-4	26
-4	230
-4	120
-4	404
-4	436
-4	156
-4	468
-4	308
-4	196
-4	288
-4	98
-4	282
-4	318
-4	318
-4	470
-4	316
-4	0
-4	490
-4	364
-4	118
-4	134
-4	282
-4	138
-4	238
-4	118
-4	72
-4	90
-4	10
-4	306
-4	224
-4	242
-4	392
-4	272
-4	242
-4	452
-4	226
-4	402
-4	396
-4	58
-4	336
-4	168
-4	34
-4	472
-4	322
-4	498
-4	160
-4	42
-4	430
-4	458
-4	78
-4	76
-4	492
-4	218
-4	228
-4	138
-4	30
-4	64
-4	468
-4	76
-4	74
-4	342
-4	230
-4	368
-4	296
-4	216
-4	344
-4	274
-4	116
-4	256
-4	70
-4	480
-4	288
-4	244
-4	438
-4	128
-4	432
-4	202
-4	316
-4	280
-4	2
-4	80
-4	44
-4	104
-4	466
-4	366
-4	406
-4	190
-4	406
-4	114
-4	258
-4	90
-4	262
-4	348
-4	424
-4	12
-4	396
-4	164
-4	454
-4	478
-4	298
-4	164
-4	424
-4	382
-4	70
-4	480
-4	24
-4	104
-4	70
-4	438
-4	414
-4	200
-4	360
-4	248
-4	444
-4	120
-4	230
-4	478
-4	178
-4	468
-4	310
-4	460
-4	480
-4	136
-4	172
-4	214
-4	462
-4	406
-4	454
-4	384
-4	256
-4	26
-4	134
-4	384
-4	18
-4	462
-4	492
-4	100
-4	298
-4	498
-4	146
-4	458
-4	362
-4	186
-4	348
-4	18
-4	344
-4	84
-4	28
-4	448
-4	152
-4	348
-4	194
-4	414
-4	222
-4	126
-4	90
-4	400
-4	200
-PREHOOK: query: select count(*) from (select key from srcpart where key % 2 = 0) t
-PREHOOK: type: QUERY
-PREHOOK: Input: default@srcpart
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
-PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
-PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
-#### A masked pattern was here ####
-POSTHOOK: query: select count(*) from (select key from srcpart where key % 2 = 0) t
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@srcpart
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
-POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
-POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
-#### A masked pattern was here ####
-988

http://git-wip-us.apache.org/repos/asf/hive/blob/0afaa8f6/ql/src/test/results/clientpositive/tez/udf_get_splits.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/udf_get_splits.q.out b/ql/src/test/results/clientpositive/tez/udf_get_splits.q.out
deleted file mode 100644
index c8ebe88..0000000
--- a/ql/src/test/results/clientpositive/tez/udf_get_splits.q.out
+++ /dev/null
@@ -1,73 +0,0 @@
-PREHOOK: query: DESCRIBE FUNCTION get_splits
-PREHOOK: type: DESCFUNCTION
-POSTHOOK: query: DESCRIBE FUNCTION get_splits
-POSTHOOK: type: DESCFUNCTION
-get_splits(string,int) - Returns an array of length int serialized splits for the referenced tables string.
-PREHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
-PREHOOK: type: DESCFUNCTION
-POSTHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
-POSTHOOK: type: DESCFUNCTION
-get_splits(string,int) - Returns an array of length int serialized splits for the referenced tables string.
-PREHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: database:default
-PREHOOK: Output: default@table_74b4da448d74412d8fc0da37a405efb3
-POSTHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@table_74b4da448d74412d8fc0da37a405efb3
-PREHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: database:default
-PREHOOK: Output: default@table_4c8d9e53ea514617a6a72158c9c843c5
-POSTHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@table_4c8d9e53ea514617a6a72158c9c843c5
-PREHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: database:default
-PREHOOK: Output: default@table_42dcc08d004e411f8038701980b491e3
-POSTHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@table_42dcc08d004e411f8038701980b491e3
-PREHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: database:default
-PREHOOK: Output: default@table_9ef6460ac4a24e4fab1ea09ad94b01e3
-POSTHOOK: query: DESCRIBE FUNCTION EXTENDED get_splits
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@table_9ef6460ac4a24e4fab1ea09ad94b01e3
-PREHOOK: query: select r.if_class as ic, r.split_class as sc, hash(r.split) as h, length(r.split) as l from (select explode(get_splits("select key, count(*) from src where key % 2 = 0 group by key",5)) as r) t
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: database:default
-PREHOOK: Output: default@table_bf88922034334ab08a9abb6cf8aa546e
-POSTHOOK: query: select r.if_class as ic, r.split_class as sc, hash(r.split) as h, length(r.split) as l from (select explode(get_splits("select key, count(*) from src where key % 2 = 0 group by key",5)) as r) t
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@table_bf88922034334ab08a9abb6cf8aa546e
-PREHOOK: query: select r.if_class as ic, r.split_class as sc, hash(r.split) as h, length(r.split) as l from (select explode(get_splits("select key, count(*) from src where key % 2 = 0 group by key",5)) as r) t
-PREHOOK: type: QUERY
-PREHOOK: Input: _dummy_database@_dummy_table
-#### A masked pattern was here ####
-POSTHOOK: query: select r.if_class as ic, r.split_class as sc, hash(r.split) as h, length(r.split) as l from (select explode(get_splits("select key, count(*) from src where key % 2 = 0 group by key",5)) as r) t
-POSTHOOK: type: QUERY
-POSTHOOK: Input: _dummy_database@_dummy_table
-#### A masked pattern was here ####
-class org.apache.hadoop.mapred.TextInputFormat	class org.apache.hadoop.mapred.FileSplit	-1434872849	218
-class org.apache.hadoop.mapred.TextInputFormat	class org.apache.hadoop.mapred.FileSplit	2107621793	218
-class org.apache.hadoop.mapred.TextInputFormat	class org.apache.hadoop.mapred.FileSplit	-1988206222	218
-class org.apache.hadoop.mapred.TextInputFormat	class org.apache.hadoop.mapred.FileSplit	1357774915	218
-class org.apache.hadoop.mapred.TextInputFormat	class org.apache.hadoop.mapred.FileSplit	605302265	218


[33/39] 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/master
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());


[14/39] hive git commit: HIVE-13419: Merge master into llap branch

Posted by jd...@apache.org.
HIVE-13419: Merge master into llap branch


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/a7b0ca73
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a7b0ca73
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a7b0ca73

Branch: refs/heads/master
Commit: a7b0ca733e416951ab6c36f71dbe512665477535
Parents: 28d1082 4e9f95a
Author: Jason Dere <jd...@hortonworks.com>
Authored: Mon Apr 4 13:37:14 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Mon Apr 4 13:37:14 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ant/GenVectorCode.java   |  531 +-
 .../org/apache/hadoop/hive/cli/CliDriver.java   |    3 +
 .../apache/hadoop/hive/common/FileUtils.java    |   54 +
 .../apache/hadoop/hive/common/ServerUtils.java  |   11 +
 .../hive/common/type/HiveIntervalDayTime.java   |  245 -
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   31 +-
 .../org/apache/hive/common/util/DateUtils.java  |   19 -
 .../hive/contrib/serde2/MultiDelimitSerDe.java  |   23 +-
 data/files/alltypesorc3xcols                    |  Bin 0 -> 1504592 bytes
 data/files/timestamps.txt                       |   50 +
 .../deployers/config/hive/hive-site.mysql.xml   |   24 +-
 .../hive/hcatalog/streaming/HiveEndPoint.java   |   11 +
 .../hcatalog/templeton/SecureProxySupport.java  |   46 +-
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     |  266 +-
 .../main/java/org/apache/hive/hplsql/Conf.java  |    2 +-
 .../main/java/org/apache/hive/hplsql/Conn.java  |    3 +-
 .../java/org/apache/hive/hplsql/Converter.java  |   15 +-
 .../main/java/org/apache/hive/hplsql/Exec.java  |  106 +-
 .../java/org/apache/hive/hplsql/Expression.java |   13 +
 .../main/java/org/apache/hive/hplsql/File.java  |   11 +
 .../main/java/org/apache/hive/hplsql/Ftp.java   |  415 +
 .../main/java/org/apache/hive/hplsql/Meta.java  |   35 +-
 .../java/org/apache/hive/hplsql/Package.java    |    3 +
 .../main/java/org/apache/hive/hplsql/Row.java   |    4 +-
 .../java/org/apache/hive/hplsql/Select.java     |    6 +-
 .../main/java/org/apache/hive/hplsql/Stmt.java  |  167 +-
 .../main/java/org/apache/hive/hplsql/Utils.java |   40 +
 .../main/java/org/apache/hive/hplsql/Var.java   |   18 +-
 .../apache/hive/hplsql/functions/Function.java  |   40 +-
 .../hive/hplsql/functions/FunctionDatetime.java |   40 +
 .../hive/hplsql/functions/FunctionString.java   |   26 +-
 .../org/apache/hive/hplsql/TestHplsqlLocal.java |   28 +-
 .../apache/hive/hplsql/TestHplsqlOffline.java   |   25 +
 .../test/queries/db/create_drop_database.sql    |    5 +
 .../queries/db/create_procedure_no_params.sql   |   25 +
 hplsql/src/test/queries/db/describe.sql         |    3 +
 hplsql/src/test/queries/db/execute.sql          |    7 +
 hplsql/src/test/queries/db/expression.sql       |    1 +
 hplsql/src/test/queries/db/for.sql              |    1 +
 hplsql/src/test/queries/db/insert.sql           |    3 +
 hplsql/src/test/queries/db/insert_directory.sql |   12 +
 hplsql/src/test/queries/db/schema.sql           |   10 +-
 hplsql/src/test/queries/db/truncate_table.sql   |    2 +
 .../src/test/queries/local/create_function3.sql |   58 +
 .../src/test/queries/local/create_function4.sql |   19 +
 .../test/queries/local/create_procedure3.sql    |   29 +
 hplsql/src/test/queries/local/declare3.sql      |    7 +
 hplsql/src/test/queries/local/if.sql            |    6 +-
 hplsql/src/test/queries/local/interval.sql      |    4 +-
 hplsql/src/test/queries/local/replace.sql       |    1 +
 .../queries/offline/create_table_mssql2.sql     |   33 +
 .../test/queries/offline/create_table_mysql.sql |    5 +
 .../test/queries/offline/create_table_ora2.sql  |    6 +
 .../test/queries/offline/create_table_pg.sql    |    5 +
 hplsql/src/test/queries/offline/update.sql      |   33 +
 .../results/db/create_drop_database.out.txt     |    8 +
 .../results/db/create_procedure_mssql.out.txt   |    2 +-
 .../db/create_procedure_no_params.out.txt       |   10 +
 .../db/create_procedure_return_cursor.out.txt   |    4 +-
 .../db/create_procedure_return_cursor2.out.txt  |    4 +-
 hplsql/src/test/results/db/describe.out.txt     |   12 +
 hplsql/src/test/results/db/execute.out.txt      |   14 +
 hplsql/src/test/results/db/expression.out.txt   |    5 +
 hplsql/src/test/results/db/for.out.txt          |   44 +
 hplsql/src/test/results/db/insert.out.txt       |    4 +
 .../test/results/db/insert_directory.out.txt    |    9 +
 .../test/results/db/rowtype_attribute.out.txt   |    2 +-
 .../src/test/results/db/truncate_table.out.txt  |    4 +
 .../test/results/local/create_function3.out.txt |   22 +
 .../test/results/local/create_function4.out.txt |    9 +
 .../test/results/local/create_package.out.txt   |    2 +-
 .../results/local/create_procedure3.out.txt     |   31 +
 .../local/create_procedure_no_params.out.txt    |   12 +-
 hplsql/src/test/results/local/declare3.out.txt  |    9 +
 hplsql/src/test/results/local/if.out.txt        |    4 +
 hplsql/src/test/results/local/interval.out.txt  |    1 +
 hplsql/src/test/results/local/replace.out.txt   |    1 +
 .../results/offline/create_table_mssql2.out.txt |   10 +
 .../results/offline/create_table_mysql.out.txt  |    4 +
 .../results/offline/create_table_ora2.out.txt   |    5 +
 .../results/offline/create_table_pg.out.txt     |    5 +
 hplsql/src/test/results/offline/update.out.txt  |   34 +
 .../listener/TestDbNotificationListener.java    |   18 +
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |   56 +-
 .../hive/metastore/TestHiveMetaStore.java       |   20 +-
 .../jdbc/TestJdbcWithLocalClusterSpark.java     |    2 +-
 .../apache/hive/jdbc/TestJdbcWithMiniMr.java    |    2 +-
 ...stMultiSessionsHS2WithLocalClusterSpark.java |    6 +-
 .../jdbc/TestServiceDiscoveryWithMiniHS2.java   |  132 +
 .../jdbc/authorization/TestHS2AuthzContext.java |    4 +-
 .../authorization/TestJdbcMetadataApiAuth.java  |    2 +-
 .../TestJdbcWithSQLAuthorization.java           |    2 +-
 .../cli/session/TestHiveSessionImpl.java        |    5 +-
 .../test/resources/testconfiguration.properties |    8 +
 ...SQLStdHiveAuthorizationValidatorForTest.java |   29 +
 .../hive/jdbc/ZooKeeperHiveClientHelper.java    |   21 +-
 .../hadoop/hive/llap/io/api/LlapProxy.java      |    2 +
 .../impl/LlapZookeeperRegistryImpl.java         |    2 +
 .../org/apache/hadoop/hive/llap/LlapUtil.java   |   26 +
 .../apache/hadoop/hive/llap/tez/Converters.java |    1 +
 .../hadoop/hive/llap/tez/TestConverters.java    |  190 +
 llap-server/bin/runLlapDaemon.sh                |    5 +-
 .../llap/IncrementalObjectSizeEstimator.java    |   54 +-
 .../hadoop/hive/llap/cache/LlapDataBuffer.java  |   12 +-
 .../hive/llap/cache/LowLevelCacheImpl.java      |   35 +-
 .../llap/cache/LowLevelCacheMemoryManager.java  |    6 +-
 .../llap/cache/LowLevelFifoCachePolicy.java     |    4 +-
 .../llap/cache/LowLevelLrfuCachePolicy.java     |   14 +-
 .../hive/llap/cache/SimpleBufferManager.java    |    8 +-
 .../hive/llap/cli/LlapOptionsProcessor.java     |    1 -
 .../hadoop/hive/llap/cli/LlapServiceDriver.java |   71 +
 .../hive/llap/daemon/impl/LlapDaemon.java       |   24 +-
 .../impl/StaticPermanentFunctionChecker.java    |   70 +
 .../hive/llap/io/api/impl/LlapInputFormat.java  |   32 +-
 .../hive/llap/io/api/impl/LlapIoImpl.java       |   21 +-
 .../llap/io/decode/OrcColumnVectorProducer.java |    4 +-
 .../llap/io/encoded/OrcEncodedDataReader.java   |   95 +-
 .../hadoop/hive/llap/old/BufferInProgress.java  |   82 -
 .../apache/hadoop/hive/llap/old/BufferPool.java |  225 -
 .../hadoop/hive/llap/old/CachePolicy.java       |   34 -
 .../apache/hadoop/hive/llap/old/ChunkPool.java  |  237 -
 .../resources/llap-daemon-log4j2.properties     |   14 +-
 .../hive/metastore/MetaStoreDirectSql.java      |   53 +-
 .../hadoop/hive/metastore/ObjectStore.java      |   10 +-
 .../hive/metastore/StatObjectConverter.java     |    2 +-
 .../hadoop/hive/metastore/hbase/HBaseStore.java |    4 +-
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |    8 +-
 .../hadoop/hive/metastore/hbase/StatsCache.java |   20 +-
 .../stats/BinaryColumnStatsAggregator.java      |   43 +-
 .../stats/BooleanColumnStatsAggregator.java     |   42 +-
 .../hbase/stats/ColumnStatsAggregator.java      |   12 +-
 .../stats/ColumnStatsAggregatorFactory.java     |    8 +-
 .../stats/DecimalColumnStatsAggregator.java     |  340 +-
 .../stats/DoubleColumnStatsAggregator.java      |  307 +-
 .../hbase/stats/IExtrapolatePartStatus.java     |   30 +
 .../hbase/stats/LongColumnStatsAggregator.java  |  305 +-
 .../stats/StringColumnStatsAggregator.java      |   85 +-
 .../hive/metastore/txn/CompactionInfo.java      |    4 +
 .../metastore/txn/CompactionTxnHandler.java     |  127 +-
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    |   18 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |  366 +-
 .../hadoop/hive/metastore/txn/TxnStore.java     |   33 +-
 .../hadoop/hive/metastore/txn/TxnUtils.java     |   99 +-
 .../metastore/txn/ValidCompactorTxnList.java    |    2 +-
 ...stHBaseAggregateStatsCacheWithBitVector.java |    6 +-
 .../TestHBaseAggregateStatsExtrapolation.java   |  717 ++
 .../TestHBaseAggregateStatsNDVUniformDist.java  |  581 ++
 .../hive/metastore/txn/TestTxnHandler.java      |   96 +-
 .../hadoop/hive/metastore/txn/TestTxnUtils.java |  135 +
 odbc/Makefile                                   |  193 -
 odbc/pom.xml                                    |  142 -
 odbc/src/cpp/HiveColumnDesc.cpp                 |  190 -
 odbc/src/cpp/HiveColumnDesc.h                   |   73 -
 odbc/src/cpp/HiveConnection.h                   |   58 -
 odbc/src/cpp/HiveResultSet.cpp                  |  616 --
 odbc/src/cpp/HiveResultSet.h                    |  190 -
 odbc/src/cpp/HiveRowSet.cpp                     |  465 --
 odbc/src/cpp/HiveRowSet.h                       |  168 -
 odbc/src/cpp/hiveclient.cpp                     |  294 -
 odbc/src/cpp/hiveclient.h                       |  598 --
 odbc/src/cpp/hiveclienthelper.cpp               |   86 -
 odbc/src/cpp/hiveclienthelper.h                 |  132 -
 odbc/src/cpp/hiveconstants.h                    |   83 -
 odbc/src/cpp/thriftserverconstants.h            |   64 -
 odbc/src/test/hiveclienttest.c                  | 1395 ----
 odbc/testdata/dataset1.input                    |    2 -
 odbc/testdata/dataset2.input                    |    1 -
 odbc/testdata/dataset_types.input               |    2 -
 orc/src/java/org/apache/orc/OrcFile.java        |   21 +-
 .../java/org/apache/orc/impl/WriterImpl.java    |   42 +-
 packaging/src/main/assembly/src.xml             |    1 -
 pom.xml                                         |    1 -
 ...eColumnArithmeticIntervalYearMonthColumn.txt |   56 +-
 ...eColumnArithmeticIntervalYearMonthScalar.txt |   55 +-
 .../DateColumnArithmeticTimestampColumn.txt     |  141 +-
 .../DateColumnArithmeticTimestampColumnBase.txt |  171 -
 .../DateColumnArithmeticTimestampScalar.txt     |  113 +-
 .../DateColumnArithmeticTimestampScalarBase.txt |  137 -
 ...eScalarArithmeticIntervalYearMonthColumn.txt |   53 +-
 .../DateScalarArithmeticTimestampColumn.txt     |  108 +-
 .../DateScalarArithmeticTimestampColumnBase.txt |  147 -
 ...ayTimeColumnCompareIntervalDayTimeColumn.txt |   52 -
 ...ayTimeColumnCompareIntervalDayTimeScalar.txt |   55 -
 ...ayTimeScalarCompareIntervalDayTimeColumn.txt |   55 -
 ...erLongDoubleColumnCompareTimestampColumn.txt |    2 +-
 ...erLongDoubleColumnCompareTimestampScalar.txt |    4 +-
 ...erLongDoubleScalarCompareTimestampColumn.txt |    4 +
 .../FilterTimestampColumnBetween.txt            |   11 +-
 ...terTimestampColumnCompareTimestampColumn.txt |  417 +-
 ...imestampColumnCompareTimestampColumnBase.txt |  429 -
 ...terTimestampColumnCompareTimestampScalar.txt |  128 +-
 ...imestampColumnCompareTimestampScalarBase.txt |  145 -
 ...erTimestampScalarCompareLongDoubleColumn.txt |    3 +-
 ...terTimestampScalarCompareTimestampColumn.txt |  132 +-
 ...imestampScalarCompareTimestampColumnBase.txt |  147 -
 ...ayTimeColumnCompareIntervalDayTimeColumn.txt |   54 -
 ...ayTimeColumnCompareIntervalDayTimeScalar.txt |   57 -
 ...ayTimeScalarCompareIntervalDayTimeColumn.txt |   57 -
 ...ervalYearMonthColumnArithmeticDateColumn.txt |   55 +-
 ...ervalYearMonthColumnArithmeticDateScalar.txt |   51 +-
 ...YearMonthColumnArithmeticTimestampColumn.txt |   63 +-
 ...YearMonthColumnArithmeticTimestampScalar.txt |   48 +-
 ...ervalYearMonthScalarArithmeticDateColumn.txt |   51 +-
 ...YearMonthScalarArithmeticTimestampColumn.txt |   55 +-
 .../LongDoubleColumnCompareTimestampColumn.txt  |    1 -
 .../LongDoubleColumnCompareTimestampScalar.txt  |    3 +-
 .../LongDoubleScalarCompareTimestampColumn.txt  |    1 +
 .../TimestampColumnArithmeticDateColumn.txt     |  138 +-
 .../TimestampColumnArithmeticDateColumnBase.txt |  172 -
 .../TimestampColumnArithmeticDateScalar.txt     |   98 +-
 .../TimestampColumnArithmeticDateScalarBase.txt |  126 -
 ...pColumnArithmeticIntervalYearMonthColumn.txt |   59 +-
 ...pColumnArithmeticIntervalYearMonthScalar.txt |   41 +-
 ...TimestampColumnArithmeticTimestampColumn.txt |  128 +-
 ...stampColumnArithmeticTimestampColumnBase.txt |  152 -
 ...TimestampColumnArithmeticTimestampScalar.txt |   96 +-
 ...stampColumnArithmeticTimestampScalarBase.txt |  125 -
 .../TimestampColumnCompareLongDoubleScalar.txt  |    1 +
 .../TimestampColumnCompareTimestampColumn.txt   |  122 +-
 ...imestampColumnCompareTimestampColumnBase.txt |  140 -
 .../TimestampColumnCompareTimestampScalar.txt   |  114 +-
 ...imestampColumnCompareTimestampScalarBase.txt |  131 -
 .../TimestampScalarArithmeticDateColumn.txt     |  117 +-
 .../TimestampScalarArithmeticDateColumnBase.txt |  151 -
 ...pScalarArithmeticIntervalYearMonthColumn.txt |   62 +-
 ...TimestampScalarArithmeticTimestampColumn.txt |  103 +-
 ...stampScalarArithmeticTimestampColumnBase.txt |  139 -
 .../TimestampScalarCompareLongDoubleColumn.txt  |    4 +-
 .../TimestampScalarCompareTimestampColumn.txt   |  115 +-
 ...imestampScalarCompareTimestampColumnBase.txt |  132 -
 .../VectorUDAFMinMaxIntervalDayTime.txt         |  454 +
 .../UDAFTemplates/VectorUDAFMinMaxTimestamp.txt |   31 +-
 .../org/apache/hadoop/hive/llap/DebugUtils.java |   43 -
 .../org/apache/hadoop/hive/llap/LogLevels.java  |   53 -
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   26 +-
 .../org/apache/hadoop/hive/ql/QueryDisplay.java |   11 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   16 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |    7 +
 .../hadoop/hive/ql/exec/OperatorUtils.java      |   45 +-
 .../hive/ql/exec/OrcFileMergeOperator.java      |    4 +-
 .../ql/exec/SparkHashTableSinkOperator.java     |    8 +-
 .../hadoop/hive/ql/exec/TableScanOperator.java  |   19 +-
 .../persistence/HybridHashTableContainer.java   |   40 +-
 .../ql/exec/persistence/KeyValueContainer.java  |   25 +-
 .../ql/exec/persistence/ObjectContainer.java    |   24 +-
 .../hive/ql/exec/persistence/RowContainer.java  |   34 +-
 .../ql/exec/spark/HiveSparkClientFactory.java   |    6 +-
 .../ql/exec/spark/SparkReduceRecordHandler.java |    2 +
 .../hadoop/hive/ql/exec/tez/DagUtils.java       |    1 +
 .../hadoop/hive/ql/exec/tez/TezJobMonitor.java  |    2 +-
 .../hive/ql/exec/vector/TimestampUtils.java     |    8 +
 .../hive/ql/exec/vector/VectorAssignRow.java    |   23 +-
 .../exec/vector/VectorColumnAssignFactory.java  |   19 +-
 .../ql/exec/vector/VectorColumnSetInfo.java     |   23 +-
 .../hive/ql/exec/vector/VectorCopyRow.java      |   32 +
 .../ql/exec/vector/VectorDeserializeRow.java    |   13 +-
 .../exec/vector/VectorExpressionDescriptor.java |    6 +-
 .../hive/ql/exec/vector/VectorExtractRow.java   |   24 +-
 .../ql/exec/vector/VectorGroupByOperator.java   |    2 +-
 .../ql/exec/vector/VectorGroupKeyHelper.java    |   12 +
 .../ql/exec/vector/VectorHashKeyWrapper.java    |   82 +-
 .../exec/vector/VectorHashKeyWrapperBatch.java  |  112 +-
 .../hive/ql/exec/vector/VectorSerializeRow.java |    7 +-
 .../ql/exec/vector/VectorizationContext.java    |   27 +-
 .../ql/exec/vector/VectorizedBatchUtil.java     |   20 +-
 .../ql/exec/vector/VectorizedRowBatchCtx.java   |   12 +-
 .../expressions/CastDecimalToTimestamp.java     |    3 +-
 .../expressions/CastDoubleToTimestamp.java      |   17 +-
 .../vector/expressions/CastLongToTimestamp.java |    8 +-
 .../CastMillisecondsLongToTimestamp.java        |   22 +-
 .../CastStringToIntervalDayTime.java            |    8 +-
 .../expressions/CastTimestampToBoolean.java     |    4 +-
 .../vector/expressions/CastTimestampToDate.java |    2 +-
 .../expressions/CastTimestampToDecimal.java     |    9 +-
 .../expressions/CastTimestampToDouble.java      |   13 +-
 .../vector/expressions/CastTimestampToLong.java |   12 +-
 .../expressions/ConstantVectorExpression.java   |   36 +-
 .../expressions/DateColSubtractDateColumn.java  |   80 +-
 .../expressions/DateColSubtractDateScalar.java  |   51 +-
 .../DateScalarSubtractDateColumn.java           |   52 +-
 .../FilterTimestampColumnInList.java            |   27 +-
 .../IfExprIntervalDayTimeColumnColumn.java      |  103 +-
 .../IfExprIntervalDayTimeColumnScalar.java      |   94 +-
 .../IfExprIntervalDayTimeScalarColumn.java      |   96 +-
 .../IfExprIntervalDayTimeScalarScalar.java      |   84 +-
 .../IfExprTimestampColumnColumnBase.java        |    8 +-
 .../IfExprTimestampColumnScalar.java            |    3 +-
 .../IfExprTimestampColumnScalarBase.java        |   14 +-
 .../IfExprTimestampScalarColumn.java            |    3 +-
 .../IfExprTimestampScalarColumnBase.java        |   15 +-
 .../IfExprTimestampScalarScalar.java            |    3 +-
 .../IfExprTimestampScalarScalarBase.java        |   13 +-
 .../expressions/LongColEqualLongScalar.java     |    2 +-
 .../LongColGreaterEqualLongScalar.java          |    2 +-
 .../expressions/LongColGreaterLongScalar.java   |    2 +-
 .../expressions/LongColLessEqualLongScalar.java |    2 +-
 .../expressions/LongColLessLongScalar.java      |    2 +-
 .../expressions/LongColNotEqualLongScalar.java  |    2 +-
 .../expressions/LongScalarEqualLongColumn.java  |    2 +-
 .../LongScalarGreaterEqualLongColumn.java       |    2 +-
 .../LongScalarGreaterLongColumn.java            |    2 +-
 .../LongScalarLessEqualLongColumn.java          |    2 +-
 .../expressions/LongScalarLessLongColumn.java   |    2 +-
 .../LongScalarNotEqualLongColumn.java           |    2 +-
 .../ql/exec/vector/expressions/NullUtil.java    |   26 +
 .../expressions/TimestampColumnInList.java      |   29 +-
 .../expressions/VectorExpressionWriter.java     |    6 +-
 .../VectorExpressionWriterFactory.java          |  124 +-
 .../expressions/VectorUDFDateAddColCol.java     |    2 +-
 .../expressions/VectorUDFDateAddColScalar.java  |    2 +-
 .../expressions/VectorUDFDateDiffColCol.java    |   10 +-
 .../expressions/VectorUDFDateDiffColScalar.java |    2 +-
 .../expressions/VectorUDFDateDiffScalarCol.java |    2 +-
 .../expressions/VectorUDFDateTimestamp.java     |    2 +-
 .../expressions/VectorUDFUnixTimeStampDate.java |    7 +-
 .../VectorUDFUnixTimeStampTimestamp.java        |    5 +-
 .../aggregates/VectorUDAFAvgTimestamp.java      |   40 +-
 .../aggregates/VectorUDAFStdPopTimestamp.java   |   24 +-
 .../aggregates/VectorUDAFStdSampTimestamp.java  |   27 +-
 .../aggregates/VectorUDAFVarPopTimestamp.java   |   24 +-
 .../aggregates/VectorUDAFVarSampTimestamp.java  |   24 +-
 .../mapjoin/VectorMapJoinCommonOperator.java    |    2 +-
 .../mapjoin/VectorMapJoinRowBytesContainer.java |   24 +-
 .../ql/exec/vector/udf/VectorUDFAdaptor.java    |   13 +-
 .../hadoop/hive/ql/io/HiveInputFormat.java      |   15 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   14 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |   15 +-
 .../hive/ql/io/orc/encoded/EncodedReader.java   |    2 +-
 .../ql/io/orc/encoded/EncodedReaderImpl.java    |  131 +-
 .../io/parquet/convert/HiveSchemaConverter.java |   10 +-
 .../parquet/read/DataWritableReadSupport.java   |   75 +-
 .../ql/io/parquet/serde/ParquetHiveSerDe.java   |   11 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  252 +-
 .../formatting/MetaDataFormatUtils.java         |    8 +-
 .../hadoop/hive/ql/optimizer/Optimizer.java     |    4 +-
 .../ql/optimizer/ReduceSinkMapJoinProc.java     |   24 +-
 .../hive/ql/optimizer/StatsOptimizer.java       |   14 +-
 .../ql/optimizer/calcite/HiveRelFactories.java  |    5 -
 .../calcite/reloperators/HiveAggregate.java     |    9 +-
 .../rules/HivePointLookupOptimizerRule.java     |  381 +
 .../ql/optimizer/pcr/PcrExprProcFactory.java    |  103 +-
 .../hive/ql/optimizer/physical/LlapDecider.java |    2 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |    7 +
 .../stats/annotation/StatsRulesProcFactory.java |    3 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   71 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   15 +-
 .../hadoop/hive/ql/parse/GenTezUtils.java       |    3 +-
 .../hadoop/hive/ql/parse/ParseContext.java      |    5 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  216 +-
 .../apache/hadoop/hive/ql/parse/TableMask.java  |  127 +
 .../org/apache/hadoop/hive/ql/plan/MapWork.java |    4 +-
 .../plugin/HiveAuthorizationValidator.java      |   11 +
 .../authorization/plugin/HiveAuthorizer.java    |   69 +
 .../plugin/HiveAuthorizerImpl.java              |   23 +
 .../authorization/plugin/HiveV1Authorizer.java  |   25 +
 .../sqlstd/DummyHiveAuthorizationValidator.java |   23 +
 .../SQLStdHiveAuthorizationValidator.java       |   22 +
 .../hadoop/hive/ql/session/SessionState.java    |    2 +-
 .../ql/txn/AcidCompactionHistoryService.java    |    7 +
 .../hive/ql/txn/AcidHouseKeeperService.java     |    7 +
 .../hadoop/hive/ql/txn/compactor/Cleaner.java   |   68 +-
 .../hive/ql/txn/compactor/CompactorThread.java  |    5 +
 .../hadoop/hive/ql/txn/compactor/Initiator.java |   28 +-
 .../hadoop/hive/ql/txn/compactor/Worker.java    |    9 +-
 .../hadoop/hive/ql/util/DateTimeMath.java       |  214 +-
 .../apache/hadoop/hive/ql/TestTxnCommands2.java |   54 +-
 .../ql/exec/persistence/TestHashPartition.java  |    3 +-
 .../TestTimestampWritableAndColumnVector.java   |   68 +
 .../TestVectorExpressionWriters.java            |   23 +-
 .../TestVectorFilterExpressions.java            |    1 -
 .../expressions/TestVectorMathFunctions.java    |   53 +-
 .../TestVectorTimestampExpressions.java         |   26 +-
 .../vector/expressions/TestVectorTypeCasts.java |  216 +-
 .../TestVectorMapJoinRowBytesContainer.java     |    3 +-
 .../FakeVectorRowBatchFromObjectIterables.java  |    3 +-
 .../vector/util/VectorizedRowGroupGenUtil.java  |   14 +-
 .../hive/ql/io/orc/TestColumnStatistics.java    |    5 +-
 .../hive/ql/io/orc/TestInputOutputFormat.java   |   97 +-
 .../hadoop/hive/ql/io/orc/TestOrcFile.java      |    5 +-
 .../hive/ql/io/orc/TestVectorOrcFile.java       |   22 +-
 .../hive/ql/lockmgr/TestDbTxnManager.java       |    6 +
 .../hive/ql/lockmgr/TestDbTxnManager2.java      |   28 +
 .../ql/optimizer/physical/TestVectorizer.java   |    5 +
 .../distinct_windowing_failure1.q               |   20 +
 .../distinct_windowing_failure2.q               |   22 +
 .../queries/clientnegative/right_side_join.q    |   12 +
 .../clientpositive/auto_join19_inclause.q       |   18 +
 .../clientpositive/bucket_map_join_tez1.q       |   27 +
 ...umn_names_with_leading_and_trailing_spaces.q |   13 +
 .../queries/clientpositive/distinct_windowing.q |   39 +
 .../queries/clientpositive/filter_in_or_dup.q   |   19 +
 ql/src/test/queries/clientpositive/llap_udf.q   |    6 +-
 ql/src/test/queries/clientpositive/masking_1.q  |   27 +
 ql/src/test/queries/clientpositive/masking_2.q  |   17 +
 ql/src/test/queries/clientpositive/masking_3.q  |   27 +
 ql/src/test/queries/clientpositive/masking_4.q  |   30 +
 ql/src/test/queries/clientpositive/masking_5.q  |   22 +
 .../clientpositive/masking_disablecbo_1.q       |   28 +
 .../clientpositive/masking_disablecbo_2.q       |   18 +
 .../clientpositive/masking_disablecbo_3.q       |   28 +
 .../clientpositive/masking_disablecbo_4.q       |   31 +
 .../clientpositive/multi_insert_with_join.q     |   29 +
 .../test/queries/clientpositive/orc_merge12.q   |   51 +
 .../clientpositive/parquet_schema_evolution.q   |   14 +
 .../sample_islocalmode_hook_use_metadata.q      |   48 +
 .../queries/clientpositive/schema_evol_stats.q  |   50 +
 .../queries/clientpositive/vector_complex_all.q |   43 +
 .../clientpositive/vector_complex_join.q        |   29 +
 .../clientpositive/vector_interval_arithmetic.q |  174 +
 .../vector_number_compare_projection.q          |  192 +
 .../clientpositive/vector_tablesample_rows.q    |   38 +
 .../clientnegative/dbtxnmgr_nodblock.q.out      |    2 +
 .../clientnegative/dbtxnmgr_nodbunlock.q.out    |    2 +
 .../distinct_windowing_failure1.q.out           |   47 +
 .../distinct_windowing_failure2.q.out           |   47 +
 .../lockneg_query_tbl_in_locked_db.q.out        |    6 +
 .../lockneg_try_db_lock_conflict.q.out          |    6 +
 .../lockneg_try_drop_locked_db.q.out            |    4 +
 .../lockneg_try_lock_db_in_use.q.out            |    6 +
 .../clientnegative/right_side_join.q.out        |    1 +
 .../clientpositive/auto_join19_inclause.q.out   |  130 +
 ...names_with_leading_and_trailing_spaces.q.out |   80 +
 .../clientpositive/constprog_semijoin.q.out     |    4 +-
 .../clientpositive/dbtxnmgr_showlocks.q.out     |    6 +-
 .../clientpositive/distinct_windowing.q.out     |  451 +
 .../dynpart_sort_optimization_acid.q.out        |    4 +-
 .../clientpositive/filter_in_or_dup.q.out       |   96 +
 .../llap/bucket_map_join_tez1.q.out             |  343 +
 .../llap/bucket_map_join_tez2.q.out             |   15 +
 .../results/clientpositive/llap/cte_3.q.out     |    4 +
 .../results/clientpositive/llap/cte_5.q.out     |    2 +
 .../results/clientpositive/llap/cte_mat_1.q.out |    2 +
 .../results/clientpositive/llap/cte_mat_2.q.out |    2 +
 .../results/clientpositive/llap/cte_mat_3.q.out |    3 +
 .../results/clientpositive/llap/cte_mat_4.q.out |    6 +
 .../results/clientpositive/llap/cte_mat_5.q.out |    3 +
 .../llap/dynamic_partition_pruning.q.out        |   95 +
 .../llap/dynamic_partition_pruning_2.q.out      |   16 +
 .../llap/hybridgrace_hashjoin_1.q.out           |   24 +
 .../llap/hybridgrace_hashjoin_2.q.out           |   36 +
 .../results/clientpositive/llap/llap_udf.q.out  |    3 +
 .../clientpositive/llap/llapdecider.q.out       |   12 +
 .../clientpositive/llap/lvj_mapjoin.q.out       |    2 +
 .../clientpositive/llap/mapjoin_decimal.q.out   |    2 +
 .../test/results/clientpositive/llap/mrr.q.out  |   10 +
 .../llap/tez_bmj_schema_evolution.q.out         |    2 +
 .../results/clientpositive/llap/tez_dml.q.out   |    3 +
 .../llap/tez_dynpart_hashjoin_1.q.out           |   12 +
 .../llap/tez_dynpart_hashjoin_2.q.out           |    9 +
 .../results/clientpositive/llap/tez_join.q.out  |    2 +
 .../clientpositive/llap/tez_join_hash.q.out     |    6 +
 .../llap/tez_join_result_complex.q.out          |    4 +
 .../clientpositive/llap/tez_join_tests.q.out    |    3 +
 .../clientpositive/llap/tez_joins_explain.q.out |    3 +
 .../clientpositive/llap/tez_self_join.q.out     |    3 +
 .../results/clientpositive/llap/tez_smb_1.q.out |    6 +
 .../clientpositive/llap/tez_smb_main.q.out      |   24 +
 .../results/clientpositive/llap/tez_union.q.out |   28 +
 .../clientpositive/llap/tez_union2.q.out        |    3 +
 .../llap/tez_union_dynamic_partition.q.out      |    2 +
 .../llap/tez_union_group_by.q.out               |    5 +
 .../llap/tez_union_multiinsert.q.out            |   13 +
 .../llap/tez_vector_dynpart_hashjoin_1.q.out    |   12 +
 .../llap/tez_vector_dynpart_hashjoin_2.q.out    |    9 +
 .../llap/vector_join_part_col_char.q.out        |    2 +
 .../vectorized_dynamic_partition_pruning.q.out  |   89 +
 .../test/results/clientpositive/masking_1.q.out |  466 ++
 .../test/results/clientpositive/masking_2.q.out |  321 +
 .../test/results/clientpositive/masking_3.q.out | 7765 ++++++++++++++++++
 .../test/results/clientpositive/masking_4.q.out |  233 +
 .../test/results/clientpositive/masking_5.q.out |  189 +
 .../clientpositive/masking_disablecbo_1.q.out   |  462 ++
 .../clientpositive/masking_disablecbo_2.q.out   |  355 +
 .../clientpositive/masking_disablecbo_3.q.out   | 7737 +++++++++++++++++
 .../clientpositive/masking_disablecbo_4.q.out   |  229 +
 .../clientpositive/multi_insert_with_join.q.out |  128 +
 .../results/clientpositive/orc_merge12.q.out    |  606 ++
 .../parquet_map_null.q.java1.8.out              |    1 +
 .../parquet_schema_evolution.q.out              |   65 +
 .../clientpositive/parquet_type_promotion.q.out |    2 +-
 .../results/clientpositive/perf/query13.q.out   |   14 +-
 .../results/clientpositive/perf/query27.q.out   |    2 +-
 .../results/clientpositive/perf/query34.q.out   |    2 +-
 .../results/clientpositive/perf/query48.q.out   |   14 +-
 .../results/clientpositive/perf/query68.q.out   |    2 +-
 .../results/clientpositive/perf/query73.q.out   |    2 +-
 .../results/clientpositive/perf/query79.q.out   |    2 +-
 .../results/clientpositive/perf/query82.q.out   |    2 +-
 .../results/clientpositive/perf/query85.q.out   |   26 +-
 .../results/clientpositive/pointlookup2.q.out   |   38 +-
 .../results/clientpositive/pointlookup3.q.out   |   50 +-
 .../results/clientpositive/pointlookup4.q.out   |    2 +-
 .../sample_islocalmode_hook_use_metadata.q.out  |  230 +
 .../clientpositive/schema_evol_stats.q.out      |  392 +
 .../spark/bucket_map_join_tez1.q.out            |  306 +
 .../spark/constprog_semijoin.q.out              |    4 +-
 .../spark/multi_insert_with_join.q.out          |  128 +
 .../tez/bucket_map_join_tez1.q.out              |  294 +
 .../clientpositive/tez/bucketpruning1.q.out     |    8 +-
 ...names_with_leading_and_trailing_spaces.q.out |   80 +
 .../clientpositive/tez/constprog_semijoin.q.out |    4 +-
 .../clientpositive/tez/explainuser_1.q.out      |   92 +-
 .../clientpositive/tez/llapdecider.q.out        |   12 +
 .../clientpositive/tez/orc_merge12.q.out        |  606 ++
 .../clientpositive/tez/schema_evol_stats.q.out  |  384 +
 .../clientpositive/tez/vector_complex_all.q.out |  254 +
 .../tez/vector_complex_join.q.out               |  227 +
 .../tez/vector_interval_arithmetic.q.out        | 1086 +++
 .../tez/vector_mr_diff_schema_alias.q.out       |    2 +-
 .../clientpositive/tez/vectorized_casts.q.out   |   18 +-
 .../tez/vectorized_timestamp.q.out              |  157 +
 .../clientpositive/vector_complex_all.q.out     |  235 +
 .../clientpositive/vector_complex_join.q.out    |  225 +
 .../vector_interval_arithmetic.q.out            | 1027 +++
 .../vector_mr_diff_schema_alias.q.out           |    2 +-
 .../vector_number_compare_projection.q.out      |  718 ++
 .../vector_tablesample_rows.q.out               |  371 +
 .../clientpositive/vectorized_casts.q.out       |   18 +-
 .../hadoop/hive/serde2/avro/AvroSerdeUtils.java |   20 +-
 .../hive/serde2/io/TimestampWritable.java       |   71 +-
 .../auth/LdapAuthenticationProviderImpl.java    |  317 +-
 .../service/cli/session/HiveSessionImpl.java    |   13 +-
 .../apache/hive/service/server/HiveServer2.java |   39 +-
 .../auth/TestLdapAtnProviderWithMiniDS.java     |  200 +-
 .../apache/hive/service/cli/CLIServiceTest.java |    8 +
 .../org/apache/hadoop/fs/ProxyFileSystem.java   |    5 +-
 .../common/io/encoded/EncodedColumnBatch.java   |    2 +-
 .../hive/common/type/HiveIntervalDayTime.java   |  253 +
 .../hadoop/hive/common/type/PisaTimestamp.java  |  609 --
 .../hadoop/hive/common/type/RandomTypeUtil.java |   70 +-
 .../hive/ql/exec/vector/ColumnVector.java       |    2 +-
 .../vector/IntervalDayTimeColumnVector.java     |  348 +
 .../ql/exec/vector/TimestampColumnVector.java   |  341 +-
 .../hive/common/util/IntervalDayTimeUtils.java  |   77 +
 .../hive/common/type/TestPisaTimestamp.java     |  118 -
 .../exec/vector/TestTimestampColumnVector.java  |  117 +
 .../ptest2/conf/example-apache-trunk.properties |    2 +-
 537 files changed, 42116 insertions(+), 12248 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a7b0ca73/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/a7b0ca73/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/a7b0ca73/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/a7b0ca73/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/a7b0ca73/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------


[16/39] hive git commit: HIVE-13461: LLAP output format service not actually registered in LLAP registry

Posted by jd...@apache.org.
HIVE-13461: LLAP output format service not actually registered in LLAP registry


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/8f6b28a3
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8f6b28a3
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8f6b28a3

Branch: refs/heads/master
Commit: 8f6b28a38d495043793585593c4fb7152bb8a112
Parents: 79c1c69
Author: Jason Dere <jd...@hortonworks.com>
Authored: Fri Apr 8 11:46:03 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Fri Apr 8 11:46:03 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java   | 1 +
 ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java       | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8f6b28a3/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
index bc99a3c..e49c047 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
@@ -259,6 +259,7 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
     srv.addInternalEndpoint(getMngEndpoint());
     srv.addInternalEndpoint(getShuffleEndpoint());
     srv.addExternalEndpoint(getServicesEndpoint());
+    srv.addInternalEndpoint(getOutputFormatEndpoint());
 
     for (Map.Entry<String, String> kv : this.conf) {
       if (kv.getKey().startsWith(HiveConf.PREFIX_LLAP)

http://git-wip-us.apache.org/repos/asf/hive/blob/8f6b28a3/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
index 30ed9cf..64e5e69 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
@@ -181,6 +181,7 @@ public class LlapRecordReader<V extends WritableComparable> implements RecordRea
           LOG.debug("Interrupting reader thread due to reader event with error " + event.getMessage());
         }
         getReaderThread().interrupt();
+        break;
       default:
         throw new RuntimeException("Unhandled ReaderEvent type " + event.getEventType() + " with message " + event.getMessage());
     }


[08/39] hive git commit: HIVE-13173: LLAP: Add end-to-end test for LlapInputFormat

Posted by jd...@apache.org.
HIVE-13173: LLAP: Add end-to-end test for LlapInputFormat


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d8a9531a
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d8a9531a
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d8a9531a

Branch: refs/heads/master
Commit: d8a9531a7738ad0137e287c4664a45b934e97c1e
Parents: f6be218
Author: Jason Dere <jd...@hortonworks.com>
Authored: Fri Feb 26 15:34:31 2016 -0800
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Fri Feb 26 15:34:31 2016 -0800

----------------------------------------------------------------------
 itests/.gitignore                               |   1 +
 .../apache/hive/jdbc/TestJdbcWithMiniLlap.java  | 203 +++++++++++++++++++
 2 files changed, 204 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d8a9531a/itests/.gitignore
----------------------------------------------------------------------
diff --git a/itests/.gitignore b/itests/.gitignore
index c2ed135..2b7798e 100644
--- a/itests/.gitignore
+++ b/itests/.gitignore
@@ -1 +1,2 @@
 thirdparty/
+hive-unit/llapdaemon*.log

http://git-wip-us.apache.org/repos/asf/hive/blob/d8a9531a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
new file mode 100644
index 0000000..88e2e55
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
@@ -0,0 +1,203 @@
+/**
+ * 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.jdbc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.lang.reflect.Field;
+import java.net.URL;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.llap.LlapRecordReader;
+import org.apache.hadoop.hive.metastore.ObjectStore;
+import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+
+import org.apache.hive.jdbc.miniHS2.MiniHS2;
+import org.apache.hive.jdbc.miniHS2.MiniHS2.MiniClusterType;
+import org.apache.hive.jdbc.LlapInputFormat;
+
+import org.datanucleus.ClassLoaderResolver;
+import org.datanucleus.NucleusContext;
+import org.datanucleus.api.jdo.JDOPersistenceManagerFactory;
+import org.datanucleus.AbstractNucleusContext;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestJdbcWithMiniLlap {
+  private static MiniHS2 miniHS2 = null;
+  private static String dataFileDir;
+  private static Path kvDataFilePath;
+  private static final String tmpDir = System.getProperty("test.tmp.dir");
+
+  private Connection hs2Conn = null;
+
+  @BeforeClass
+  public static void beforeTest() throws Exception {
+    Class.forName(MiniHS2.getJdbcDriverName());
+
+    String confDir = "../../data/conf/llap/";
+    if (confDir != null && !confDir.isEmpty()) {
+      HiveConf.setHiveSiteLocation(new URL("file://"+ new File(confDir).toURI().getPath() + "/hive-site.xml"));
+      System.out.println("Setting hive-site: "+HiveConf.getHiveSiteLocation());
+    }
+
+    HiveConf conf = new HiveConf();
+    conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    // Necessary for GetSplits()/LlapInputFormat,
+    // the config generated for the query fragment needs to include the MapWork
+    conf.setBoolVar(HiveConf.ConfVars.HIVE_RPC_QUERY_PLAN, true);
+
+    conf.addResource(new URL("file://" + new File(confDir).toURI().getPath()
+        + "/tez-site.xml"));
+    conf.addResource(new URL("file://" + new File(confDir).toURI().getPath()
+        + "/llap-daemon-site.xml"));
+
+    miniHS2 = new MiniHS2(conf, MiniClusterType.LLAP, true);
+
+    dataFileDir = conf.get("test.data.files").replace('\\', '/').replace("c:", "");
+    kvDataFilePath = new Path(dataFileDir, "kv1.txt");
+    Map<String, String> confOverlay = new HashMap<String, String>();
+    miniHS2.start(confOverlay);
+    miniHS2.getDFS().getFileSystem().mkdirs(new Path("/apps_staging_dir/anonymous"));
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    hs2Conn = getConnection(miniHS2.getJdbcURL(), System.getProperty("user.name"), "bar");
+  }
+
+  private Connection getConnection(String jdbcURL, String user, String pwd) throws SQLException {
+    Connection conn = DriverManager.getConnection(jdbcURL, user, pwd);
+    conn.createStatement().execute("set hive.support.concurrency = false");
+    return conn;
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    hs2Conn.close();
+  }
+
+  @AfterClass
+  public static void afterTest() throws Exception {
+    if (miniHS2.isStarted()) {
+      miniHS2.stop();
+    }
+  }
+
+  private void createTestTable(String tableName) throws Exception {
+    Statement stmt = hs2Conn.createStatement();
+
+    // create table
+    stmt.execute("DROP TABLE IF EXISTS " + tableName);
+    stmt.execute("CREATE TABLE " + tableName
+        + " (under_col INT COMMENT 'the under column', value STRING) COMMENT ' test table'");
+
+    // load data
+    stmt.execute("load data local inpath '"
+        + kvDataFilePath.toString() + "' into table " + tableName);
+
+    ResultSet res = stmt.executeQuery("SELECT * FROM " + tableName);
+    assertTrue(res.next());
+    assertEquals("val_238", res.getString(2));
+    res.close();
+    stmt.close();
+  }
+
+  @Test
+  public void testLlapInputFormatEndToEnd() throws Exception {
+    createTestTable("testtab1");
+    String url = miniHS2.getJdbcURL();
+    String user = System.getProperty("user.name");
+    String pwd = user;
+    String query = "select * from testtab1 where under_col = 0";
+
+    LlapInputFormat inputFormat = new LlapInputFormat(url, user, pwd, query);
+    JobConf job = new JobConf();
+    int numSplits = 1;
+
+    InputSplit[] splits = inputFormat.getSplits(job, numSplits);
+    assert(splits.length > 0);
+
+    boolean first = true;
+    int rowCount = 0;
+    for (InputSplit split : splits) {
+      System.out.println("Processing split " + split.getLocations());
+
+      RecordReader<NullWritable, Text> reader = inputFormat.getRecordReader(split, job, null);
+      if (reader instanceof LlapRecordReader && first) {
+        Schema schema = ((LlapRecordReader)reader).getSchema();
+        System.out.println(""+schema);
+      }
+
+      if (first) {
+        System.out.println("Results: ");
+        System.out.println("");
+        first = false;
+      }
+
+      Text value = reader.createValue();
+      while (reader.next(NullWritable.get(), value)) {
+        System.out.println(value);
+        ++rowCount;
+      }
+    }
+    assertEquals(3, rowCount);
+  }
+}
\ No newline at end of file


[21/39] hive git commit: Merge branch 'master' into llap

Posted by jd...@apache.org.
Merge branch 'master' into llap

Conflicts:
	llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
	llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/99cb7f96
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/99cb7f96
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/99cb7f96

Branch: refs/heads/master
Commit: 99cb7f96f1a516698b587320e841d26cd7bb034d
Parents: 0afaa8f 2d28291
Author: Jason Dere <jd...@hortonworks.com>
Authored: Tue Apr 19 15:24:19 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Tue Apr 19 15:24:19 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/BeeLine.java   |   22 +-
 .../hive/beeline/BeeLineCommandCompleter.java   |   26 +-
 .../beeline/SeparatedValuesOutputFormat.java    |    2 +-
 bin/ext/cleardanglingscratchdir.cmd             |   35 +
 bin/ext/cleardanglingscratchdir.sh              |   28 +
 bin/ext/llapstatus.sh                           |   42 +
 .../apache/hadoop/hive/common/FileUtils.java    |    2 +-
 .../hadoop/hive/common/type/HiveBaseChar.java   |    6 +
 .../hadoop/hive/common/type/HiveVarchar.java    |    4 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   11 +-
 .../org/apache/hadoop/hive/conf/Validator.java  |    2 +-
 .../apache/hadoop/hive/ql/log/PerfLogger.java   |   51 +-
 .../apache/hive/common/util/StreamPrinter.java  |   18 +-
 .../java/org/apache/hive/http/HttpServer.java   |   29 +-
 .../hive/common/type/TestHiveBaseChar.java      |    2 +
 .../hadoop/hive/hbase/HBaseSerDeParameters.java |    6 +-
 .../hive/hbase/HiveHBaseInputFormatUtil.java    |    6 +-
 .../test/results/positive/hbase_pushdown.q.out  |    2 +-
 .../test/results/positive/ppd_key_ranges.q.out  |    2 +-
 .../mapreduce/FileOutputCommitterContainer.java |    4 +-
 .../mapreduce/FosterStorageHandler.java         |    6 +-
 .../streaming/StreamingIntegrationTester.java   |   12 +-
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |   15 +-
 .../hive/metastore/TestMetaStoreMetrics.java    |   13 +-
 .../TestHiveAuthorizerCheckInvocation.java      |    2 +-
 .../plugin/TestHiveAuthorizerShowFilters.java   |    4 +-
 .../ql/session/TestClearDanglingScratchDir.java |  131 +
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |   78 +
 .../apache/hive/jdbc/TestJdbcWithMiniHA.java    |  200 ++
 .../jdbc/authorization/TestHS2AuthzContext.java |   14 +-
 .../authorization/TestJdbcMetadataApiAuth.java  |    4 +-
 .../hive/jdbc/miniHS2/StartMiniHS2Cluster.java  |    2 +-
 .../hive/jdbc/miniHS2/TestHs2Metrics.java       |   41 +-
 .../service/cli/session/TestQueryDisplay.java   |    2 +
 .../cli/thrift/TestThriftBinaryCLIService.java  |   92 -
 .../cli/thrift/TestThriftHttpCLIService.java    |  241 --
 .../thrift/ThriftCliServiceMessageSizeTest.java |  140 +
 itests/qtest/pom.xml                            |    2 +-
 .../hive/ql/security/DummyAuthenticator.java    |    5 +
 .../security/InjectableDummyAuthenticator.java  |    5 +
 ...SQLStdHiveAuthorizationValidatorForTest.java |   47 +-
 .../org/apache/hive/jdbc/HiveBaseResultSet.java |   10 +-
 .../org/apache/hive/jdbc/HiveConnection.java    |   84 +-
 .../org/apache/hive/jdbc/HiveStatement.java     |   18 +
 jdbc/src/java/org/apache/hive/jdbc/Utils.java   |   20 +-
 .../hive/llap/registry/ServiceInstance.java     |    6 +
 .../hive/llap/registry/ServiceRegistry.java     |    4 +-
 .../registry/impl/LlapFixedRegistryImpl.java    |   29 +-
 .../llap/registry/impl/LlapRegistryService.java |   17 +-
 .../impl/LlapZookeeperRegistryImpl.java         |   29 +-
 llap-server/pom.xml                             |   55 +
 llap-server/sql/serviceCheckScript.sql          |   12 +
 .../hive/llap/cli/LlapOptionsProcessor.java     |   23 +-
 .../hadoop/hive/llap/cli/LlapServiceDriver.java |    8 +
 .../llap/cli/LlapStatusOptionsProcessor.java    |  139 +
 .../hive/llap/cli/LlapStatusServiceDriver.java  |  821 ++++++
 .../hive/llap/daemon/impl/LlapDaemon.java       |   17 +-
 .../daemon/services/impl/LlapWebServices.java   |  176 +-
 .../llap/io/decode/OrcEncodedDataConsumer.java  |   45 +-
 .../llap/metrics/LlapDaemonCacheMetrics.java    |    4 +-
 .../llap/metrics/LlapDaemonExecutorMetrics.java |    4 +-
 .../llap/metrics/LlapDaemonQueueMetrics.java    |    4 +-
 .../hadoop/hive/llap/metrics/MetricsUtils.java  |    1 -
 .../hive/llap/security/SecretManager.java       |   18 +-
 .../llap/shufflehandler/ShuffleHandler.java     |    2 +-
 .../hadoop-metrics2.properties.template         |   50 +
 .../main/resources/llap-cli-log4j2.properties   |    9 +-
 .../hive/llap/daemon/MiniLlapCluster.java       |    5 +-
 .../hive/metastore/MetaStoreSchemaInfo.java     |    4 +-
 .../hadoop/hive/metastore/MetaStoreUtils.java   |   28 +-
 .../hive/metastore/hbase/HBaseImport.java       |    4 +-
 .../hive/metastore/hbase/HBaseReadWrite.java    |   10 +-
 orc/src/java/org/apache/orc/OrcUtils.java       |   75 +
 orc/src/java/org/apache/orc/Reader.java         |    6 +
 orc/src/java/org/apache/orc/RecordReader.java   |    8 +-
 .../java/org/apache/orc/TypeDescription.java    |   62 +-
 .../org/apache/orc/impl/BitFieldReader.java     |    5 +-
 .../java/org/apache/orc/impl/IntegerReader.java |   26 +-
 .../apache/orc/impl/RunLengthByteReader.java    |   36 +-
 .../apache/orc/impl/RunLengthIntegerReader.java |   31 +-
 .../orc/impl/RunLengthIntegerReaderV2.java      |   33 +-
 .../java/org/apache/orc/impl/WriterImpl.java    |   47 +-
 packaging/src/main/assembly/bin.xml             |    9 +
 pom.xml                                         |    1 +
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   22 +-
 .../hive/ql/exec/AbstractFileMergeOperator.java |    9 +
 .../hive/ql/exec/AppMasterEventOperator.java    |    4 +-
 .../hadoop/hive/ql/exec/CollectOperator.java    |   11 +-
 .../hadoop/hive/ql/exec/CommonJoinOperator.java |    8 +-
 .../hadoop/hive/ql/exec/DemuxOperator.java      |    8 +-
 .../hadoop/hive/ql/exec/DummyStoreOperator.java |   11 +-
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |    4 +-
 .../hadoop/hive/ql/exec/FilterOperator.java     |    4 +-
 .../hadoop/hive/ql/exec/ForwardOperator.java    |    4 +-
 .../hadoop/hive/ql/exec/GroupByOperator.java    |    4 +-
 .../hive/ql/exec/HashTableDummyOperator.java    |    4 +-
 .../ql/exec/LateralViewForwardOperator.java     |    5 +-
 .../hive/ql/exec/LateralViewJoinOperator.java   |    4 +-
 .../hadoop/hive/ql/exec/LimitOperator.java      |    4 +-
 .../hadoop/hive/ql/exec/ListSinkOperator.java   |   11 +-
 .../apache/hadoop/hive/ql/exec/MapOperator.java |    9 +-
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |    1 +
 .../apache/hadoop/hive/ql/exec/MuxOperator.java |    8 +-
 .../apache/hadoop/hive/ql/exec/Operator.java    |   22 +-
 .../apache/hadoop/hive/ql/exec/PTFOperator.java |    4 +-
 .../hadoop/hive/ql/exec/ScriptOperator.java     |   40 +-
 .../hadoop/hive/ql/exec/SelectOperator.java     |    4 +-
 .../ql/exec/SparkHashTableSinkOperator.java     |   10 +-
 .../hadoop/hive/ql/exec/TableScanOperator.java  |    2 +-
 .../apache/hadoop/hive/ql/exec/TaskRunner.java  |    3 +-
 .../hadoop/hive/ql/exec/TerminalOperator.java   |   10 +
 .../hadoop/hive/ql/exec/UDTFOperator.java       |    8 +-
 .../hadoop/hive/ql/exec/UnionOperator.java      |    2 +-
 .../hadoop/hive/ql/exec/mr/MapRedTask.java      |    5 +-
 .../hadoop/hive/ql/exec/mr/MapredLocalTask.java |    9 +-
 .../vector/VectorAppMasterEventOperator.java    |   11 +-
 .../ql/exec/vector/VectorFileSinkOperator.java  |    8 +-
 .../ql/exec/vector/VectorFilterOperator.java    |    8 +-
 .../ql/exec/vector/VectorGroupByOperator.java   |   25 +-
 .../ql/exec/vector/VectorLimitOperator.java     |    5 +-
 .../ql/exec/vector/VectorMapJoinOperator.java   |   11 +-
 .../VectorMapJoinOuterFilteredOperator.java     |    8 +-
 .../hive/ql/exec/vector/VectorMapOperator.java  |    5 +-
 .../exec/vector/VectorReduceSinkOperator.java   |    8 +-
 .../exec/vector/VectorSMBMapJoinOperator.java   |   13 +-
 .../ql/exec/vector/VectorSelectOperator.java    |   21 +-
 .../VectorSparkHashTableSinkOperator.java       |    6 +-
 ...VectorSparkPartitionPruningSinkOperator.java |    8 +-
 .../ql/exec/vector/VectorizationContext.java    |   11 +-
 .../ql/exec/vector/VectorizedRowBatchCtx.java   |   13 +-
 .../hadoop/hive/ql/hooks/HookContext.java       |   11 +-
 .../hive/ql/index/IndexPredicateAnalyzer.java   |   35 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   43 +-
 .../hive/ql/io/orc/OrcRawRecordMerger.java      |    7 +-
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |   12 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |   50 +-
 .../hadoop/hive/ql/io/orc/SchemaEvolution.java  |  234 +-
 .../hive/ql/io/orc/TreeReaderFactory.java       |  858 +++---
 .../ql/io/orc/VectorizedOrcInputFormat.java     |   32 +-
 .../hadoop/hive/ql/io/orc/WriterImpl.java       |    2 -
 .../ql/io/parquet/convert/ETypeConverter.java   |    3 +-
 .../io/parquet/convert/HiveStructConverter.java |    2 +-
 .../write/ParquetRecordWriterWrapper.java       |    4 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |    2 +-
 .../optimizer/ConstantPropagateProcFactory.java |   22 +-
 .../hadoop/hive/ql/optimizer/IndexUtils.java    |    2 +
 .../SizeBasedBigTableSelectorForAutoSMJ.java    |    2 +-
 .../hive/ql/optimizer/StatsOptimizer.java       |   37 +-
 .../calcite/cost/HiveAlgorithmsUtil.java        |   12 +-
 .../hive/ql/optimizer/physical/LlapDecider.java |   12 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |    2 +
 .../stats/annotation/StatsRulesProcFactory.java |  107 +-
 .../apache/hadoop/hive/ql/parse/ASTNode.java    |    2 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |    4 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |    4 +-
 .../hadoop/hive/ql/parse/MaskAndFilterInfo.java |   38 +
 .../apache/hadoop/hive/ql/parse/ParseUtils.java |    8 +-
 .../hadoop/hive/ql/parse/ReplicationSpec.java   |    3 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   53 +-
 .../apache/hadoop/hive/ql/parse/TableMask.java  |   85 +-
 .../hadoop/hive/ql/parse/TableSample.java       |    4 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |   84 +-
 .../hive/ql/parse/spark/GenSparkUtils.java      |    2 +-
 .../SparkPartitionPruningSinkOperator.java      |   16 +-
 .../hadoop/hive/ql/plan/ExprNodeDescUtils.java  |   14 +-
 .../apache/hadoop/hive/ql/plan/PlanUtils.java   |    2 +-
 .../hadoop/hive/ql/processors/CommandUtil.java  |    6 +-
 .../ql/security/HadoopDefaultAuthenticator.java |    5 +
 .../ql/security/HiveAuthenticationProvider.java |    2 +
 .../SessionStateConfigUserAuthenticator.java    |    5 +
 .../security/SessionStateUserAuthenticator.java |    5 +
 .../AuthorizationMetaStoreFilterHook.java       |    4 +-
 .../plugin/HiveAuthorizationValidator.java      |   12 +-
 .../authorization/plugin/HiveAuthorizer.java    |   68 +-
 .../plugin/HiveAuthorizerImpl.java              |   20 +-
 .../authorization/plugin/HiveAuthzContext.java  |   83 -
 .../plugin/HivePrivilegeObject.java             |   31 +
 .../authorization/plugin/HiveV1Authorizer.java  |   20 +-
 .../authorization/plugin/QueryContext.java      |   78 +
 .../sqlstd/DummyHiveAuthorizationValidator.java |   21 +-
 .../SQLStdHiveAuthorizationValidator.java       |   22 +-
 .../ql/session/ClearDanglingScratchDir.java     |  176 ++
 .../hadoop/hive/ql/session/OperationLog.java    |   10 +-
 .../hadoop/hive/ql/session/SessionState.java    |   43 +-
 .../hive/ql/stats/fs/FSStatsAggregator.java     |    2 +-
 .../hive/ql/stats/fs/FSStatsPublisher.java      |    3 +-
 .../hive/ql/txn/compactor/CompactorMR.java      |   12 +-
 .../org/apache/hadoop/hive/ql/udf/UDFSign.java  |   15 +
 .../apache/hadoop/hive/ql/udf/UDFToByte.java    |    7 +-
 .../apache/hadoop/hive/ql/udf/UDFToDouble.java  |    6 +-
 .../apache/hadoop/hive/ql/udf/UDFToFloat.java   |    6 +-
 .../apache/hadoop/hive/ql/udf/UDFToInteger.java |    4 +
 .../apache/hadoop/hive/ql/udf/UDFToLong.java    |    4 +
 .../apache/hadoop/hive/ql/udf/UDFToShort.java   |    4 +
 .../ql/udf/generic/GenericUDAFComputeStats.java |    6 +
 .../hive/ql/udf/generic/GenericUDFBetween.java  |    2 +-
 .../hive/ql/exec/TestFunctionRegistry.java      |   26 +-
 .../hadoop/hive/ql/exec/TestOperatorNames.java  |   98 +
 .../vector/util/FakeCaptureOutputOperator.java  |   10 +-
 .../util/FakeVectorDataSourceOperator.java      |   11 +-
 .../hive/ql/io/orc/TestTypeDescription.java     |    4 +-
 .../hive/ql/io/orc/TestVectorOrcFile.java       | 1634 ++++++------
 .../hive/ql/io/orc/TestVectorizedORCReader.java |    7 +-
 .../hive/ql/metadata/TestTableIterable.java     |   67 +
 .../hive/ql/parse/TestSemanticAnalyzer.java     |   37 +
 .../ql/udf/generic/TestGenericUDFOPMinus.java   |    4 +-
 .../udf/generic/TestGenericUDFOPMultiply.java   |    4 +-
 .../ql/udf/generic/TestGenericUDFOPPlus.java    |    4 +-
 .../clientpositive/alter_partition_change_col.q |    1 +
 .../clientpositive/alter_table_cascade.q        |    1 +
 .../clientpositive/cbo_rp_udf_udaf_stats_opt.q  |   22 +
 .../queries/clientpositive/float_equality.q     |    3 +
 ql/src/test/queries/clientpositive/foldts.q     |   20 +
 .../test/queries/clientpositive/type_widening.q |    6 +
 .../queries/clientpositive/vector_between_in.q  |   30 +
 .../vector_orc_string_reader_empty_dict.q       |   20 +
 .../queries/clientpositive/windowing_gby2.q     |   41 +
 .../results/clientnegative/dyn_part_max.q.out   |    2 +-
 .../clientpositive/annotate_stats_filter.q.out  |    2 +-
 .../results/clientpositive/auto_join16.q.out    |    4 +-
 .../results/clientpositive/auto_join4.q.out     |    2 +-
 .../results/clientpositive/auto_join5.q.out     |    2 +-
 .../results/clientpositive/auto_join8.q.out     |    2 +-
 .../auto_join_reordering_values.q.out           |    2 +-
 .../clientpositive/auto_sortmerge_join_8.q.out  |    2 +
 .../test/results/clientpositive/cbo_const.q.out |    2 +-
 .../results/clientpositive/cbo_rp_join1.q.out   |    4 +-
 .../cbo_rp_udf_udaf_stats_opt.q.out             |  126 +
 .../clientpositive/constprog_semijoin.q.out     |   24 +-
 .../clientpositive/correlationoptimizer13.q.out |    2 +-
 .../clientpositive/correlationoptimizer9.q.out  |    4 +-
 .../clientpositive/dynamic_rdd_cache.q.out      |    8 +-
 .../dynpart_sort_optimization.q.out             |    2 +-
 .../clientpositive/explain_logical.q.out        |    6 +-
 .../clientpositive/filter_cond_pushdown.q.out   |    8 +-
 .../results/clientpositive/float_equality.q.out |   17 +
 ql/src/test/results/clientpositive/foldts.q.out |  154 ++
 .../groupby_multi_single_reducer3.q.out         |    8 +-
 .../clientpositive/groupby_position.q.out       |    4 +-
 .../identity_project_remove_skip.q.out          |    2 +-
 .../clientpositive/index_auto_mult_tables.q.out |   20 +-
 .../index_auto_mult_tables_compact.q.out        |   20 +-
 .../clientpositive/index_auto_self_join.q.out   |   20 +-
 .../results/clientpositive/index_bitmap3.q.out  |    4 +-
 .../clientpositive/index_bitmap_auto.q.out      |    4 +-
 .../index_bitmap_compression.q.out              |    4 +-
 .../clientpositive/infer_const_type.q.out       |   10 +-
 .../clientpositive/input_testxpath4.q.out       |    2 +-
 ql/src/test/results/clientpositive/join16.q.out |    4 +-
 ql/src/test/results/clientpositive/join19.q.out |    8 +-
 ql/src/test/results/clientpositive/join4.q.out  |    2 +-
 ql/src/test/results/clientpositive/join42.q.out |    2 +-
 ql/src/test/results/clientpositive/join5.q.out  |    2 +-
 ql/src/test/results/clientpositive/join8.q.out  |    2 +-
 .../clientpositive/join_grp_diff_keys.q.out     |    2 +-
 .../results/clientpositive/join_reorder2.q.out  |    2 +-
 .../results/clientpositive/join_reorder3.q.out  |    2 +-
 .../llap/dynamic_partition_pruning.q.out        |   20 +-
 .../llap/dynamic_partition_pruning_2.q.out      |   30 +-
 .../llap/hybridgrace_hashjoin_2.q.out           |   12 +-
 .../clientpositive/llap/tez_join_hash.q.out     |    4 +
 .../llap/tez_union_group_by.q.out               |    2 +-
 .../vectorized_dynamic_partition_pruning.q.out  |   20 +-
 .../clientpositive/louter_join_ppr.q.out        |    4 +-
 .../test/results/clientpositive/masking_1.q.out |   12 +-
 .../test/results/clientpositive/masking_2.q.out |   10 +-
 .../test/results/clientpositive/masking_3.q.out |    2 +-
 .../test/results/clientpositive/masking_4.q.out |    4 +-
 .../test/results/clientpositive/masking_5.q.out |    2 +-
 .../clientpositive/masking_disablecbo_1.q.out   |   14 +-
 .../clientpositive/masking_disablecbo_2.q.out   |   10 +-
 .../clientpositive/masking_disablecbo_3.q.out   |    2 +-
 .../clientpositive/masking_disablecbo_4.q.out   |    4 +-
 .../results/clientpositive/multiMapJoin1.q.out  |   16 +-
 .../clientpositive/orc_predicate_pushdown.q.out |   24 +-
 .../parquet_predicate_pushdown.q.out            |   28 +-
 ql/src/test/results/clientpositive/pcs.q.out    |    6 +-
 .../results/clientpositive/perf/query13.q.out   |    6 +-
 .../results/clientpositive/perf/query15.q.out   |    2 +-
 .../results/clientpositive/perf/query17.q.out   |   14 +-
 .../results/clientpositive/perf/query18.q.out   |    6 +-
 .../results/clientpositive/perf/query19.q.out   |    4 +-
 .../results/clientpositive/perf/query21.q.out   |    2 +-
 .../results/clientpositive/perf/query22.q.out   |    2 +-
 .../results/clientpositive/perf/query25.q.out   |   12 +-
 .../results/clientpositive/perf/query26.q.out   |    4 +-
 .../results/clientpositive/perf/query27.q.out   |    4 +-
 .../results/clientpositive/perf/query29.q.out   |   18 +-
 .../results/clientpositive/perf/query31.q.out   |   12 +-
 .../results/clientpositive/perf/query32.q.out   |    2 +-
 .../results/clientpositive/perf/query34.q.out   |    6 +-
 .../results/clientpositive/perf/query39.q.out   |    8 +-
 .../results/clientpositive/perf/query40.q.out   |    2 +-
 .../results/clientpositive/perf/query42.q.out   |    2 +-
 .../results/clientpositive/perf/query45.q.out   |    4 +-
 .../results/clientpositive/perf/query46.q.out   |   14 +-
 .../results/clientpositive/perf/query48.q.out   |    6 +-
 .../results/clientpositive/perf/query50.q.out   |    6 +-
 .../results/clientpositive/perf/query52.q.out   |    2 +-
 .../results/clientpositive/perf/query54.q.out   |   10 +-
 .../results/clientpositive/perf/query55.q.out   |    2 +-
 .../results/clientpositive/perf/query64.q.out   |   16 +-
 .../results/clientpositive/perf/query65.q.out   |    4 +-
 .../results/clientpositive/perf/query66.q.out   |    4 +-
 .../results/clientpositive/perf/query67.q.out   |    2 +-
 .../results/clientpositive/perf/query68.q.out   |    4 +-
 .../results/clientpositive/perf/query7.q.out    |    4 +-
 .../results/clientpositive/perf/query71.q.out   |   12 +-
 .../results/clientpositive/perf/query72.q.out   |    6 +-
 .../results/clientpositive/perf/query73.q.out   |    6 +-
 .../results/clientpositive/perf/query75.q.out   |   14 +-
 .../results/clientpositive/perf/query76.q.out   |    6 +-
 .../results/clientpositive/perf/query79.q.out   |    4 +-
 .../results/clientpositive/perf/query80.q.out   |    6 +-
 .../results/clientpositive/perf/query82.q.out   |    4 +-
 .../results/clientpositive/perf/query84.q.out   |    4 +-
 .../results/clientpositive/perf/query85.q.out   |   10 +-
 .../results/clientpositive/perf/query88.q.out   |   32 +-
 .../results/clientpositive/perf/query89.q.out   |   16 +-
 .../results/clientpositive/perf/query90.q.out   |    4 +-
 .../results/clientpositive/perf/query91.q.out   |    8 +-
 .../results/clientpositive/perf/query92.q.out   |    4 +-
 .../results/clientpositive/perf/query93.q.out   |    2 +-
 .../results/clientpositive/perf/query94.q.out   |    2 +-
 .../results/clientpositive/perf/query95.q.out   |    2 +-
 .../results/clientpositive/perf/query96.q.out   |    4 +-
 .../results/clientpositive/pointlookup.q.out    |   12 +-
 .../results/clientpositive/pointlookup2.q.out   |   16 +-
 .../results/clientpositive/pointlookup3.q.out   |    8 +-
 .../results/clientpositive/ppd_gby_join.q.out   |    8 +-
 .../test/results/clientpositive/ppd_join.q.out  |    8 +-
 .../test/results/clientpositive/ppd_join2.q.out |   12 +-
 .../test/results/clientpositive/ppd_join3.q.out |   12 +-
 .../test/results/clientpositive/ppd_join4.q.out |    2 +-
 .../clientpositive/ppd_outer_join2.q.out        |    8 +-
 .../clientpositive/ppd_outer_join3.q.out        |    8 +-
 .../clientpositive/ppd_outer_join4.q.out        |   12 +-
 .../results/clientpositive/ppd_udf_case.q.out   |    4 +-
 .../test/results/clientpositive/ppd_union.q.out |    8 +-
 ql/src/test/results/clientpositive/ppd_vc.q.out |    2 +-
 .../clientpositive/rcfile_null_value.q.out      |    2 +-
 .../clientpositive/router_join_ppr.q.out        |    4 +-
 .../test/results/clientpositive/sample8.q.out   |    4 +-
 .../test/results/clientpositive/semijoin.q.out  |    2 +-
 .../test/results/clientpositive/semijoin2.q.out |    4 +-
 .../test/results/clientpositive/semijoin4.q.out |    4 +-
 .../clientpositive/skewjoin_mapjoin9.q.out      |    4 +-
 .../results/clientpositive/skewjoinopt12.q.out  |    8 +-
 .../results/clientpositive/skewjoinopt14.q.out  |    4 +-
 .../results/clientpositive/skewjoinopt16.q.out  |    8 +-
 .../results/clientpositive/skewjoinopt17.q.out  |    8 +-
 .../results/clientpositive/skewjoinopt2.q.out   |   16 +-
 .../results/clientpositive/smb_mapjoin_10.q.out |    2 +-
 .../results/clientpositive/smb_mapjoin_14.q.out |    2 +-
 .../clientpositive/sort_merge_join_desc_2.q.out |    2 +-
 .../clientpositive/sort_merge_join_desc_3.q.out |    2 +-
 .../clientpositive/sort_merge_join_desc_4.q.out |    4 +-
 .../clientpositive/sort_merge_join_desc_8.q.out |    4 +-
 .../clientpositive/spark/auto_join16.q.out      |    4 +-
 .../clientpositive/spark/auto_join4.q.out       |    2 +-
 .../clientpositive/spark/auto_join5.q.out       |    2 +-
 .../clientpositive/spark/auto_join8.q.out       |    2 +-
 .../spark/auto_join_reordering_values.q.out     |    2 +-
 .../spark/constprog_semijoin.q.out              |   24 +-
 .../spark/dynamic_rdd_cache.q.out               |    8 +-
 .../spark/groupby_multi_single_reducer3.q.out   |    8 +-
 .../clientpositive/spark/groupby_position.q.out |    4 +-
 .../spark/identity_project_remove_skip.q.out    |    2 +-
 .../spark/index_auto_self_join.q.out            |   12 +-
 .../clientpositive/spark/index_bitmap3.q.out    |    4 +-
 .../spark/index_bitmap_auto.q.out               |    4 +-
 .../results/clientpositive/spark/join16.q.out   |    4 +-
 .../results/clientpositive/spark/join19.q.out   |    8 +-
 .../results/clientpositive/spark/join4.q.out    |    2 +-
 .../results/clientpositive/spark/join5.q.out    |    2 +-
 .../results/clientpositive/spark/join8.q.out    |    2 +-
 .../clientpositive/spark/join_reorder2.q.out    |    2 +-
 .../clientpositive/spark/join_reorder3.q.out    |    2 +-
 .../clientpositive/spark/louter_join_ppr.q.out  |    4 +-
 .../clientpositive/spark/ppd_gby_join.q.out     |    8 +-
 .../results/clientpositive/spark/ppd_join.q.out |    8 +-
 .../clientpositive/spark/ppd_join2.q.out        |   12 +-
 .../clientpositive/spark/ppd_join3.q.out        |   12 +-
 .../clientpositive/spark/ppd_outer_join2.q.out  |    8 +-
 .../clientpositive/spark/ppd_outer_join3.q.out  |    8 +-
 .../clientpositive/spark/ppd_outer_join4.q.out  |   12 +-
 .../clientpositive/spark/router_join_ppr.q.out  |    4 +-
 .../results/clientpositive/spark/sample8.q.out  |    4 +-
 .../results/clientpositive/spark/semijoin.q.out |    2 +-
 .../clientpositive/spark/skewjoinopt12.q.out    |    8 +-
 .../clientpositive/spark/skewjoinopt14.q.out    |    4 +-
 .../clientpositive/spark/skewjoinopt16.q.out    |    8 +-
 .../clientpositive/spark/skewjoinopt17.q.out    |    8 +-
 .../clientpositive/spark/skewjoinopt2.q.out     |   16 +-
 .../clientpositive/spark/smb_mapjoin_10.q.out   |    2 +-
 .../clientpositive/spark/smb_mapjoin_14.q.out   |    2 +-
 .../spark/sort_merge_join_desc_2.q.out          |   22 +-
 .../spark/sort_merge_join_desc_3.q.out          |   22 +-
 .../spark/sort_merge_join_desc_4.q.out          |   26 +-
 .../spark/sort_merge_join_desc_8.q.out          |   42 +-
 .../spark/spark_dynamic_partition_pruning.q.out | 2313 +++++++++-------
 ...k_vectorized_dynamic_partition_pruning.q.out | 2515 +++++++++++-------
 .../clientpositive/spark/subquery_in.q.out      |    2 +-
 .../spark/vector_between_in.q.out               |  332 +++
 .../spark/vector_mapjoin_reduce.q.out           |    6 +-
 .../clientpositive/spark/vectorization_0.q.out  |    2 +-
 .../clientpositive/spark/vectorization_14.q.out |    2 +-
 .../clientpositive/spark/vectorization_17.q.out |    2 +-
 .../spark/vectorization_short_regress.q.out     |    2 +-
 .../spark/vectorized_string_funcs.q.out         |    2 +-
 .../results/clientpositive/subquery_in.q.out    |    2 +-
 .../clientpositive/tez/bucketpruning1.q.out     |   32 +-
 .../clientpositive/tez/constprog_semijoin.q.out |   24 +-
 .../tez/dynamic_partition_pruning.q.out         |   20 +-
 .../tez/dynamic_partition_pruning_2.q.out       |   30 +-
 .../tez/dynpart_sort_optimization.q.out         |    2 +-
 .../clientpositive/tez/explainuser_1.q.out      |   38 +-
 .../clientpositive/tez/explainuser_2.q.out      |   12 +-
 .../clientpositive/tez/explainuser_3.q.out      |    8 +-
 .../tez/hybridgrace_hashjoin_2.q.out            |   12 +-
 .../clientpositive/tez/subquery_in.q.out        |    2 +-
 .../clientpositive/tez/tez_join_hash.q.out      |    4 +
 .../clientpositive/tez/tez_union_group_by.q.out |    2 +-
 .../tez/vector_aggregate_without_gby.q.out      |    6 +-
 .../tez/vector_auto_smb_mapjoin_14.q.out        |   38 +-
 .../clientpositive/tez/vector_between_in.q.out  |  336 +++
 .../clientpositive/tez/vector_date_1.q.out      |    4 +-
 .../tez/vector_decimal_cast.q.out               |    2 +-
 .../tez/vector_decimal_expressions.q.out        |    2 +-
 .../tez/vector_groupby_mapjoin.q.out            |    4 +-
 .../clientpositive/tez/vector_interval_2.q.out  |   12 +-
 .../tez/vector_join_part_col_char.q.out         |   10 +-
 .../tez/vector_leftsemi_mapjoin.q.out           |   12 +-
 .../tez/vector_mapjoin_reduce.q.out             |    6 +-
 .../clientpositive/tez/vectorization_0.q.out    |    2 +-
 .../clientpositive/tez/vectorization_14.q.out   |    2 +-
 .../clientpositive/tez/vectorization_17.q.out   |    2 +-
 .../clientpositive/tez/vectorization_7.q.out    |    4 +-
 .../tez/vectorization_short_regress.q.out       |    2 +-
 .../vectorized_dynamic_partition_pruning.q.out  |   20 +-
 .../tez/vectorized_parquet_types.q.out          |    2 +-
 .../tez/vectorized_string_funcs.q.out           |    2 +-
 .../tez/vectorized_timestamp.q.out              |    8 +-
 .../results/clientpositive/type_widening.q.out  |  112 +
 .../results/clientpositive/udf_greatest.q.out   |    4 +-
 .../test/results/clientpositive/udf_least.q.out |    4 +-
 .../clientpositive/udf_to_unix_timestamp.q.out  |    2 +-
 .../clientpositive/vector_between_in.q.out      |  304 +++
 .../results/clientpositive/vector_date_1.q.out  |    4 +-
 .../clientpositive/vector_decimal_cast.q.out    |    2 +-
 .../vector_decimal_expressions.q.out            |    2 +-
 .../clientpositive/vector_interval_2.q.out      |   12 +-
 .../vector_leftsemi_mapjoin.q.out               |   12 +-
 .../clientpositive/vector_mapjoin_reduce.q.out  |    6 +-
 .../vector_orc_string_reader_empty_dict.q.out   |   62 +
 .../clientpositive/vectorization_0.q.out        |    2 +-
 .../clientpositive/vectorization_14.q.out       |    2 +-
 .../clientpositive/vectorization_17.q.out       |    2 +-
 .../clientpositive/vectorization_7.q.out        |    4 +-
 .../vectorization_short_regress.q.out           |    2 +-
 .../vectorized_string_funcs.q.out               |    2 +-
 .../results/clientpositive/windowing_gby2.q.out |  652 +++++
 .../serde2/MetadataTypedColumnsetSerDe.java     |    2 +-
 .../serde2/dynamic_type/thrift_grammar.java     |    2 +-
 .../hive/serde2/lazy/LazySerDeParameters.java   |    3 +-
 .../hadoop/hive/serde2/lazy/LazyUtils.java      |    2 +-
 .../primitive/JavaHiveCharObjectInspector.java  |   15 +-
 .../JavaHiveVarcharObjectInspector.java         |   15 +-
 .../PrimitiveObjectInspectorConverter.java      |    8 +-
 .../serde2/thrift/TCTLSeparatedProtocol.java    |   25 +-
 .../hive/serde2/typeinfo/TypeInfoUtils.java     |   17 +-
 .../TestObjectInspectorConverters.java          |   95 +
 .../TestStandardObjectInspectors.java           |   14 +-
 .../org/apache/hive/tmpl/QueryProfileTmpl.jamon |   16 +-
 .../hive/service/cli/HiveSQLException.java      |    6 +-
 .../cli/operation/MetadataOperation.java        |    6 +-
 .../hive/service/cli/operation/Operation.java   |    2 +-
 .../service/cli/operation/SQLOperation.java     |    5 +
 .../cli/operation/SQLOperationDisplay.java      |    9 +
 .../service/cli/session/HiveSessionImpl.java    |    2 +
 .../service/cli/session/SessionManager.java     |   21 +-
 .../cli/thrift/ThriftBinaryCLIService.java      |   64 +-
 .../service/cli/thrift/ThriftCLIService.java    |   61 +-
 .../service/cli/thrift/ThriftHttpServlet.java   |   17 +
 .../apache/hive/service/server/HiveServer2.java |    4 +-
 .../hive-webapps/hiveserver2/hiveserver2.jsp    |   22 +-
 .../cli/TestRetryingThriftCLIServiceClient.java |  130 +-
 .../cli/thrift/ThriftCLIServiceTest.java        |    4 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |   34 +-
 .../apache/hadoop/hive/shims/HadoopShims.java   |    6 +
 .../hive/ql/exec/vector/BytesColumnVector.java  |   11 +
 .../ql/exec/vector/TimestampColumnVector.java   |    2 +-
 .../hive/ql/exec/vector/UnionColumnVector.java  |    2 -
 .../junit/runners/ConcurrentTestRunner.java     |    4 +-
 494 files changed, 11925 insertions(+), 5644 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/99cb7f96/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/99cb7f96/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstance.java
----------------------------------------------------------------------
diff --cc llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstance.java
index a504146,7e37e96..9004d3c
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstance.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/ServiceInstance.java
@@@ -53,14 -53,13 +53,20 @@@ public interface ServiceInstance 
     */
    public int getShufflePort();
  
+ 
+   /**
+    * Address for services hosted on http
+    * @return
+    */
+   public String getServicesAddress();
    /**
 +   * OutputFormat endpoint for service instance
 +   *
 +   * @return
 +   */
 +  public int getOutputFormatPort();
 +
 +  /**
     * Return the last known state (without refreshing)
     * 
     * @return

http://git-wip-us.apache.org/repos/asf/hive/blob/99cb7f96/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
----------------------------------------------------------------------
diff --cc llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
index 33ab591,bd814b9..4536a6e
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
@@@ -52,7 -54,8 +54,9 @@@ public class LlapFixedRegistryImpl impl
    private final int port;
    private final int shuffle;
    private final int mngPort;
+   private final int webPort;
 +  private final int outputFormatPort;
+   private final String webScheme;
    private final String[] hosts;
    private final int memory;
    private final int vcores;
@@@ -66,8 -69,12 +70,13 @@@
      this.shuffle = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_YARN_SHUFFLE_PORT);
      this.resolveHosts = conf.getBoolean(FIXED_REGISTRY_RESOLVE_HOST_NAMES, true);
      this.mngPort = HiveConf.getIntVar(conf, ConfVars.LLAP_MANAGEMENT_RPC_PORT);
 +    this.outputFormatPort = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
  
+ 
+     this.webPort = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_WEB_PORT);
+     boolean isSsl = HiveConf.getBoolVar(conf, ConfVars.LLAP_DAEMON_WEB_SSL);
+     this.webScheme = isSsl ? "https" : "http";
+ 
      for (Map.Entry<String, String> kv : conf) {
        if (kv.getKey().startsWith(HiveConf.PREFIX_LLAP)
            || kv.getKey().startsWith(HiveConf.PREFIX_HIVE_LLAP)) {
@@@ -153,11 -171,11 +173,16 @@@
      }
  
      @Override
 +    public int getOutputFormatPort() {
 +      return LlapFixedRegistryImpl.this.outputFormatPort;
 +    }
 +
 +    @Override
+     public String getServicesAddress() {
+       return serviceAddress;
+     }
+ 
+     @Override
      public boolean isAlive() {
        return true;
      }

http://git-wip-us.apache.org/repos/asf/hive/blob/99cb7f96/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------
diff --cc llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
index e49c047,6af30d4..9de4d17
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
@@@ -246,13 -238,8 +246,13 @@@ public class LlapZookeeperRegistryImpl 
          HiveConf.getIntVar(conf, ConfVars.LLAP_MANAGEMENT_RPC_PORT)));
    }
  
 +  public Endpoint getOutputFormatEndpoint() {
 +    return RegistryTypeUtils.ipcEndpoint(IPC_OUTPUTFORMAT, new InetSocketAddress(hostname,
 +        HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT)));
 +  }
 +
    @Override
-   public void register() throws IOException {
+   public String register() throws IOException {
      ServiceRecord srv = new ServiceRecord();
      Endpoint rpcEndpoint = getRpcEndpoint();
      srv.addInternalEndpoint(rpcEndpoint);
@@@ -323,7 -310,7 +324,8 @@@
      private final int rpcPort;
      private final int mngPort;
      private final int shufflePort;
 +    private final int outputFormatPort;
+     private final String serviceAddress;
  
      public DynamicServiceInstance(ServiceRecord srv) throws IOException {
        this.srv = srv;
@@@ -331,23 -322,22 +337,26 @@@
        final Endpoint shuffle = srv.getInternalEndpoint(IPC_SHUFFLE);
        final Endpoint rpc = srv.getInternalEndpoint(IPC_LLAP);
        final Endpoint mng = srv.getInternalEndpoint(IPC_MNG);
 +      final Endpoint outputFormat = srv.getInternalEndpoint(IPC_OUTPUTFORMAT);
+       final Endpoint services = srv.getExternalEndpoint(IPC_SERVICES);
  
        this.host =
            RegistryTypeUtils.getAddressField(rpc.addresses.get(0),
                AddressTypes.ADDRESS_HOSTNAME_FIELD);
        this.rpcPort =
-           Integer.valueOf(RegistryTypeUtils.getAddressField(rpc.addresses.get(0),
+           Integer.parseInt(RegistryTypeUtils.getAddressField(rpc.addresses.get(0),
                AddressTypes.ADDRESS_PORT_FIELD));
        this.mngPort =
-           Integer.valueOf(RegistryTypeUtils.getAddressField(mng.addresses.get(0),
+           Integer.parseInt(RegistryTypeUtils.getAddressField(mng.addresses.get(0),
                AddressTypes.ADDRESS_PORT_FIELD));
        this.shufflePort =
-           Integer.valueOf(RegistryTypeUtils.getAddressField(shuffle.addresses.get(0),
+           Integer.parseInt(RegistryTypeUtils.getAddressField(shuffle.addresses.get(0),
                AddressTypes.ADDRESS_PORT_FIELD));
 +      this.outputFormatPort =
 +          Integer.valueOf(RegistryTypeUtils.getAddressField(outputFormat.addresses.get(0),
 +              AddressTypes.ADDRESS_PORT_FIELD));
+       this.serviceAddress =
+           RegistryTypeUtils.getAddressField(services.addresses.get(0), AddressTypes.ADDRESS_URI);
      }
  
      @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/99cb7f96/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/99cb7f96/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/99cb7f96/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/99cb7f96/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/99cb7f96/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
----------------------------------------------------------------------


[22/39] hive git commit: HIVE-13558: Update LlapDump

Posted by jd...@apache.org.
HIVE-13558: Update LlapDump


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/5816ff30
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/5816ff30
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/5816ff30

Branch: refs/heads/master
Commit: 5816ff3038ca6b263d97e5b7e8f10f97f2b657ec
Parents: 99cb7f9
Author: Jason Dere <jd...@hortonworks.com>
Authored: Wed Apr 20 12:46:53 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Wed Apr 20 12:46:53 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/llap/LlapDump.java   | 60 +++++++++++++++++---
 1 file changed, 51 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/5816ff30/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
----------------------------------------------------------------------
diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
index ce419af..1c4397f 100644
--- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
+++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
@@ -24,8 +24,11 @@ import java.io.IOException;
 import java.io.FileInputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
+import java.util.Properties;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -49,10 +52,11 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
-import org.apache.hadoop.hive.llap.LlapBaseRecordReader;
-import org.apache.hadoop.hive.llap.Schema;
-
 import org.apache.hadoop.hive.llap.LlapBaseInputFormat;
+import org.apache.hadoop.hive.llap.LlapRowInputFormat;
+import org.apache.hadoop.hive.llap.LlapRowRecordReader;
+import org.apache.hadoop.hive.llap.Row;
+import org.apache.hadoop.hive.llap.Schema;
 
 public class LlapDump {
 
@@ -90,6 +94,8 @@ public class LlapDump {
       numSplits = cli.getOptionValue("n");
     }
 
+    Properties configProps = cli.getOptionProperties("hiveconf");
+
     if (cli.getArgs().length > 0) {
       query = cli.getArgs()[0];
     }
@@ -98,8 +104,18 @@ public class LlapDump {
     System.out.println("user: "+user);
     System.out.println("query: "+query);
 
-    LlapBaseInputFormat format = new LlapBaseInputFormat(url, user, pwd, query);
+    LlapRowInputFormat format = new LlapRowInputFormat();
+
     JobConf job = new JobConf();
+    job.set(LlapBaseInputFormat.URL_KEY, url);
+    job.set(LlapBaseInputFormat.USER_KEY, user);
+    job.set(LlapBaseInputFormat.PWD_KEY, pwd);
+    job.set(LlapBaseInputFormat.QUERY_KEY, query);
+
+    // Additional conf settings specified on the command line
+    for (String key: configProps.stringPropertyNames()) {
+      job.set(key, configProps.getProperty(key));
+    }
 
     InputSplit[] splits = format.getSplits(job, Integer.parseInt(numSplits));
 
@@ -111,10 +127,10 @@ public class LlapDump {
 
       for (InputSplit s: splits) {
         LOG.info("Processing input split s from " + Arrays.toString(s.getLocations()));
-        RecordReader<NullWritable, Text> reader = format.getRecordReader(s, job, null);
+        RecordReader<NullWritable, Row> reader = format.getRecordReader(s, job, null);
 
-        if (reader instanceof LlapBaseRecordReader && first) {
-          Schema schema = ((LlapBaseRecordReader)reader).getSchema();
+        if (reader instanceof LlapRowRecordReader && first) {
+          Schema schema = ((LlapRowRecordReader)reader).getSchema();
           System.out.println(""+schema);
         }
 
@@ -124,15 +140,27 @@ public class LlapDump {
           first = false;
         }
 
-        Text value = reader.createValue();
+        Row value = reader.createValue();
         while (reader.next(NullWritable.get(), value)) {
-          System.out.println(value);
+          printRow(value);
         }
       }
       System.exit(0);
     }
   }
 
+  private static void printRow(Row row) {
+    Schema schema = row.getSchema();
+    StringBuilder sb = new StringBuilder();
+    for (int idx = 0; idx < schema.getColumns().size(); ++idx) {
+      if (idx > 0) {
+        sb.append(", ");
+        sb.append(row.getValue(idx));
+      }
+    }
+    System.out.println(sb.toString());
+  }
+
   static Options createOptions() {
     Options result = new Options();
 
@@ -160,6 +188,20 @@ public class LlapDump {
         .hasArg()
         .create('n'));
 
+    result.addOption(OptionBuilder
+        .withValueSeparator()
+        .hasArgs(2)
+        .withArgName("property=value")
+        .withLongOpt("hiveconf")
+        .withDescription("Use value for given property")
+        .create());
+
+    result.addOption(OptionBuilder
+        .withLongOpt("help")
+        .withDescription("help")
+        .hasArg(false)
+        .create('h'));
+
     return result;
   }
 }


[27/39] hive git commit: Merge branch 'master' into llap

Posted by jd...@apache.org.
Merge branch 'master' into llap

Conflicts:
	common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
	llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/9f999f25
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/9f999f25
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/9f999f25

Branch: refs/heads/master
Commit: 9f999f252746a11c766624ca947e31f3fe59ec07
Parents: 390cb8c 9e1fa0c
Author: Jason Dere <jd...@hortonworks.com>
Authored: Sat Apr 30 18:30:25 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Sat Apr 30 18:30:25 2016 -0700

----------------------------------------------------------------------
 HIVE-13509.2.patch                              | 478 ++++++++++++
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  10 +-
 dev-support/jenkins-common.sh                   |  14 +
 dev-support/jenkins-submit-build.sh             |   4 +
 .../hive/hcatalog/common/HCatConstants.java     |   3 +
 .../hcatalog/mapreduce/HCatBaseInputFormat.java |  29 +-
 .../hive/hcatalog/pig/TestHCatLoader.java       |  55 ++
 .../service/cli/session/TestQueryDisplay.java   |   4 +-
 .../test/resources/testconfiguration.properties |   1 +
 .../impl/LlapZookeeperRegistryImpl.java         |  74 +-
 .../hive/llap/tezplugins/ContainerFactory.java  |   3 +-
 .../tezplugins/LlapTaskSchedulerService.java    | 377 ++++++++--
 .../llap/tezplugins/helpers/MonotonicClock.java |  24 +
 .../scheduler/LoggingFutureCallback.java        |  44 ++
 .../TestLlapTaskSchedulerService.java           | 734 ++++++++++++++++++-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   2 +-
 .../metastore/TestHiveMetaStoreGetMetaConf.java | 151 ++++
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  13 +-
 .../org/apache/hadoop/hive/ql/QueryDisplay.java |  19 +-
 .../org/apache/hadoop/hive/ql/QueryPlan.java    |  16 +-
 .../hadoop/hive/ql/exec/ConditionalTask.java    |   1 +
 .../apache/hadoop/hive/ql/exec/Registry.java    | 323 +++++---
 .../org/apache/hadoop/hive/ql/exec/Task.java    |  14 +-
 .../hadoop/hive/ql/exec/mr/ExecDriver.java      |   2 +-
 .../hive/ql/exec/mr/HadoopJobExecHelper.java    |   6 +-
 .../hadoop/hive/ql/exec/mr/MapredLocalTask.java |  16 +-
 .../hadoop/hive/ql/exec/tez/DagUtils.java       |   1 -
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  39 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   2 +-
 .../hadoop/hive/ql/session/SessionState.java    |   3 +-
 .../clientpositive/auto_sortmerge_join_8.q.out  |   2 -
 .../results/clientpositive/groupby1_limit.q.out |   2 +-
 .../clientpositive/llap/tez_join_hash.q.out     |   4 -
 .../clientpositive/tez/tez_join_hash.q.out      |   4 -
 .../apache/hive/service/cli/CLIServiceTest.java |   4 +-
 .../apache/hadoop/hive/shims/ShimLoader.java    |  10 +-
 36 files changed, 2172 insertions(+), 316 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9f999f25/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --cc common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index d7afa4d,b13de92..7db492f
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@@ -2806,8 -2808,9 +2811,11 @@@ public class HiveConf extends Configura
          false,
          "Whether to setup split locations to match nodes on which llap daemons are running," +
              " instead of using the locations provided by the split itself"),
+     LLAP_VALIDATE_ACLS("hive.llap.validate.acls", true,
+         "Whether LLAP should reject permissive ACLs in some cases (e.g. its own management\n" +
+         "protocol or ZK paths), similar to how ssh refuses a key with bad access permissions."),
 +    LLAP_DAEMON_OUTPUT_SERVICE_PORT("hive.llap.daemon.output.service.port", 15003,
 +        "LLAP daemon output service port"),
  
      SPARK_CLIENT_FUTURE_TIMEOUT("hive.spark.client.future.timeout",
        "60s", new TimeValidator(TimeUnit.SECONDS),

http://git-wip-us.apache.org/repos/asf/hive/blob/9f999f25/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------
diff --cc llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
index 0dc7599,6981061..fde70e7
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
@@@ -87,8 -89,8 +89,9 @@@ public class LlapZookeeperRegistryImpl 
    private static final String IPC_MNG = "llapmng";
    private static final String IPC_SHUFFLE = "shuffle";
    private static final String IPC_LLAP = "llap";
 +  private static final String IPC_OUTPUTFORMAT = "llapoutputformat";
    private final static String ROOT_NAMESPACE = "llap";
+   private final static String USER_SCOPE_PATH_PREFIX = "user-";
  
    private final Configuration conf;
    private final CuratorFramework zooKeeperClient;
@@@ -173,7 -177,8 +178,8 @@@
      // worker does not respond due to communication interruptions it will retain the same sequence
      // number when it returns back. If session timeout expires, the node will be deleted and new
      // addition of the same node (restart) will get next sequence number
-     this.pathPrefix = "/" + getZkPathUser(this.conf) + "/" + instanceName + "/workers/worker-";
 -    this.userPathPrefix = USER_SCOPE_PATH_PREFIX + RegistryUtils.currentUser();
++    this.userPathPrefix = USER_SCOPE_PATH_PREFIX + getZkPathUser(this.conf);
+     this.pathPrefix = "/" + userPathPrefix + "/" + instanceName + "/workers/worker-";
      this.instancesCache = null;
      this.instances = null;
      this.stateChangeListeners = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/hive/blob/9f999f25/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
----------------------------------------------------------------------


[05/39] hive git commit: HIVE-13162: Fixes for LlapDump and FileSinkoperator

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/57761e34/ql/src/test/results/clientpositive/llap/udtf_get_splits.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/udtf_get_splits.q.out b/ql/src/test/results/clientpositive/llap/udtf_get_splits.q.out
new file mode 100644
index 0000000..2f17a91
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/udtf_get_splits.q.out
@@ -0,0 +1,2130 @@
+PREHOOK: query: DESCRIBE FUNCTION get_splits
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESCRIBE FUNCTION get_splits
+POSTHOOK: type: DESCFUNCTION
+get_splits(string,int) - Returns an array of length int serialized splits for the referenced tables string.
+PREHOOK: query: DESCRIBE FUNCTION execute_splits
+PREHOOK: type: DESCFUNCTION
+POSTHOOK: query: DESCRIBE FUNCTION execute_splits
+POSTHOOK: type: DESCFUNCTION
+execute_splits(string,int) - Returns an array of length int serialized splits for the referenced tables string.
+PREHOOK: query: select r1, r2, floor(length(r3)/100000)
+from
+  (select
+    get_splits(
+      "select key, count(*) from srcpart where key % 2 = 0 group by key",
+      5) as (r1, r2, r3)) t
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select r1, r2, floor(length(r3)/100000)
+from
+  (select
+    get_splits(
+      "select key, count(*) from srcpart where key % 2 = 0 group by key",
+      5) as (r1, r2, r3)) t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+PREHOOK: query: select r1, r2, floor(length(r3)/100000)
+from
+  (select
+    get_splits(
+      "select key, count(*) from srcpart where key % 2 = 0 group by key",
+      5) as (r1, r2, r3)) t
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Output: database:default
+PREHOOK: Output: default@#### A masked pattern was here ####
+POSTHOOK: query: select r1, r2, floor(length(r3)/100000)
+from
+  (select
+    get_splits(
+      "select key, count(*) from srcpart where key % 2 = 0 group by key",
+      5) as (r1, r2, r3)) t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@#### A masked pattern was here ####
+org.apache.hadoop.hive.llap.LlapInputFormat	org.apache.hadoop.hive.llap.LlapInputSplit	1
+PREHOOK: query: select r1, r2, floor(length(r3)/100000)
+from
+  (select
+    get_splits(
+      "select key from srcpart where key % 2 = 0",
+      5) as (r1, r2, r3)) t
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select r1, r2, floor(length(r3)/100000)
+from
+  (select
+    get_splits(
+      "select key from srcpart where key % 2 = 0",
+      5) as (r1, r2, r3)) t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+org.apache.hadoop.hive.llap.LlapInputFormat	org.apache.hadoop.hive.llap.LlapInputSplit	1
+org.apache.hadoop.hive.llap.LlapInputFormat	org.apache.hadoop.hive.llap.LlapInputSplit	1
+org.apache.hadoop.hive.llap.LlapInputFormat	org.apache.hadoop.hive.llap.LlapInputSplit	1
+org.apache.hadoop.hive.llap.LlapInputFormat	org.apache.hadoop.hive.llap.LlapInputSplit	1
+PREHOOK: query: show tables
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:default
+POSTHOOK: query: show tables
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:default
+alltypesorc
+cbo_t1
+cbo_t2
+cbo_t3
+lineitem
+part
+src
+src1
+src_cbo
+src_json
+src_sequencefile
+src_thrift
+srcbucket
+srcbucket2
+srcpart
+#### A masked pattern was here ####
+PREHOOK: query: select r1, r2
+from
+  (select
+    execute_splits(
+      "select key from srcpart where key % 2 = 0",
+      1) as (r1, r2)) t
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select r1, r2
+from
+  (select
+    execute_splits(
+      "select key from srcpart where key % 2 = 0",
+      1) as (r1, r2)) t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+1	238
+1	86
+1	278
+1	98
+1	484
+1	150
+1	224
+1	66
+1	128
+1	146
+1	406
+1	374
+1	152
+1	82
+1	166
+1	430
+1	252
+1	292
+1	338
+1	446
+1	394
+1	482
+1	174
+1	494
+1	466
+1	208
+1	174
+1	396
+1	162
+1	266
+1	342
+1	0
+1	128
+1	316
+1	302
+1	438
+1	170
+1	20
+1	378
+1	92
+1	72
+1	4
+1	280
+1	208
+1	356
+1	382
+1	498
+1	386
+1	192
+1	286
+1	176
+1	54
+1	138
+1	216
+1	430
+1	278
+1	176
+1	318
+1	332
+1	180
+1	284
+1	12
+1	230
+1	260
+1	404
+1	384
+1	272
+1	138
+1	84
+1	348
+1	466
+1	58
+1	8
+1	230
+1	208
+1	348
+1	24
+1	172
+1	42
+1	158
+1	496
+1	0
+1	322
+1	468
+1	454
+1	100
+1	298
+1	418
+1	96
+1	26
+1	230
+1	120
+1	404
+1	436
+1	156
+1	468
+1	308
+1	196
+1	288
+1	98
+1	282
+1	318
+1	318
+1	470
+1	316
+1	0
+1	490
+1	364
+1	118
+1	134
+1	282
+1	138
+1	238
+1	118
+1	72
+1	90
+1	10
+1	306
+1	224
+1	242
+1	392
+1	272
+1	242
+1	452
+1	226
+1	402
+1	396
+1	58
+1	336
+1	168
+1	34
+1	472
+1	322
+1	498
+1	160
+1	42
+1	430
+1	458
+1	78
+1	76
+1	492
+1	218
+1	228
+1	138
+1	30
+1	64
+1	468
+1	76
+1	74
+1	342
+1	230
+1	368
+1	296
+1	216
+1	344
+1	274
+1	116
+1	256
+1	70
+1	480
+1	288
+1	244
+1	438
+1	128
+1	432
+1	202
+1	316
+1	280
+1	2
+1	80
+1	44
+1	104
+1	466
+1	366
+1	406
+1	190
+1	406
+1	114
+1	258
+1	90
+1	262
+1	348
+1	424
+1	12
+1	396
+1	164
+1	454
+1	478
+1	298
+1	164
+1	424
+1	382
+1	70
+1	480
+1	24
+1	104
+1	70
+1	438
+1	414
+1	200
+1	360
+1	248
+1	444
+1	120
+1	230
+1	478
+1	178
+1	468
+1	310
+1	460
+1	480
+1	136
+1	172
+1	214
+1	462
+1	406
+1	454
+1	384
+1	256
+1	26
+1	134
+1	384
+1	18
+1	462
+1	492
+1	100
+1	298
+1	498
+1	146
+1	458
+1	362
+1	186
+1	348
+1	18
+1	344
+1	84
+1	28
+1	448
+1	152
+1	348
+1	194
+1	414
+1	222
+1	126
+1	90
+1	400
+1	200
+2	238
+2	86
+2	278
+2	98
+2	484
+2	150
+2	224
+2	66
+2	128
+2	146
+2	406
+2	374
+2	152
+2	82
+2	166
+2	430
+2	252
+2	292
+2	338
+2	446
+2	394
+2	482
+2	174
+2	494
+2	466
+2	208
+2	174
+2	396
+2	162
+2	266
+2	342
+2	0
+2	128
+2	316
+2	302
+2	438
+2	170
+2	20
+2	378
+2	92
+2	72
+2	4
+2	280
+2	208
+2	356
+2	382
+2	498
+2	386
+2	192
+2	286
+2	176
+2	54
+2	138
+2	216
+2	430
+2	278
+2	176
+2	318
+2	332
+2	180
+2	284
+2	12
+2	230
+2	260
+2	404
+2	384
+2	272
+2	138
+2	84
+2	348
+2	466
+2	58
+2	8
+2	230
+2	208
+2	348
+2	24
+2	172
+2	42
+2	158
+2	496
+2	0
+2	322
+2	468
+2	454
+2	100
+2	298
+2	418
+2	96
+2	26
+2	230
+2	120
+2	404
+2	436
+2	156
+2	468
+2	308
+2	196
+2	288
+2	98
+2	282
+2	318
+2	318
+2	470
+2	316
+2	0
+2	490
+2	364
+2	118
+2	134
+2	282
+2	138
+2	238
+2	118
+2	72
+2	90
+2	10
+2	306
+2	224
+2	242
+2	392
+2	272
+2	242
+2	452
+2	226
+2	402
+2	396
+2	58
+2	336
+2	168
+2	34
+2	472
+2	322
+2	498
+2	160
+2	42
+2	430
+2	458
+2	78
+2	76
+2	492
+2	218
+2	228
+2	138
+2	30
+2	64
+2	468
+2	76
+2	74
+2	342
+2	230
+2	368
+2	296
+2	216
+2	344
+2	274
+2	116
+2	256
+2	70
+2	480
+2	288
+2	244
+2	438
+2	128
+2	432
+2	202
+2	316
+2	280
+2	2
+2	80
+2	44
+2	104
+2	466
+2	366
+2	406
+2	190
+2	406
+2	114
+2	258
+2	90
+2	262
+2	348
+2	424
+2	12
+2	396
+2	164
+2	454
+2	478
+2	298
+2	164
+2	424
+2	382
+2	70
+2	480
+2	24
+2	104
+2	70
+2	438
+2	414
+2	200
+2	360
+2	248
+2	444
+2	120
+2	230
+2	478
+2	178
+2	468
+2	310
+2	460
+2	480
+2	136
+2	172
+2	214
+2	462
+2	406
+2	454
+2	384
+2	256
+2	26
+2	134
+2	384
+2	18
+2	462
+2	492
+2	100
+2	298
+2	498
+2	146
+2	458
+2	362
+2	186
+2	348
+2	18
+2	344
+2	84
+2	28
+2	448
+2	152
+2	348
+2	194
+2	414
+2	222
+2	126
+2	90
+2	400
+2	200
+3	238
+3	86
+3	278
+3	98
+3	484
+3	150
+3	224
+3	66
+3	128
+3	146
+3	406
+3	374
+3	152
+3	82
+3	166
+3	430
+3	252
+3	292
+3	338
+3	446
+3	394
+3	482
+3	174
+3	494
+3	466
+3	208
+3	174
+3	396
+3	162
+3	266
+3	342
+3	0
+3	128
+3	316
+3	302
+3	438
+3	170
+3	20
+3	378
+3	92
+3	72
+3	4
+3	280
+3	208
+3	356
+3	382
+3	498
+3	386
+3	192
+3	286
+3	176
+3	54
+3	138
+3	216
+3	430
+3	278
+3	176
+3	318
+3	332
+3	180
+3	284
+3	12
+3	230
+3	260
+3	404
+3	384
+3	272
+3	138
+3	84
+3	348
+3	466
+3	58
+3	8
+3	230
+3	208
+3	348
+3	24
+3	172
+3	42
+3	158
+3	496
+3	0
+3	322
+3	468
+3	454
+3	100
+3	298
+3	418
+3	96
+3	26
+3	230
+3	120
+3	404
+3	436
+3	156
+3	468
+3	308
+3	196
+3	288
+3	98
+3	282
+3	318
+3	318
+3	470
+3	316
+3	0
+3	490
+3	364
+3	118
+3	134
+3	282
+3	138
+3	238
+3	118
+3	72
+3	90
+3	10
+3	306
+3	224
+3	242
+3	392
+3	272
+3	242
+3	452
+3	226
+3	402
+3	396
+3	58
+3	336
+3	168
+3	34
+3	472
+3	322
+3	498
+3	160
+3	42
+3	430
+3	458
+3	78
+3	76
+3	492
+3	218
+3	228
+3	138
+3	30
+3	64
+3	468
+3	76
+3	74
+3	342
+3	230
+3	368
+3	296
+3	216
+3	344
+3	274
+3	116
+3	256
+3	70
+3	480
+3	288
+3	244
+3	438
+3	128
+3	432
+3	202
+3	316
+3	280
+3	2
+3	80
+3	44
+3	104
+3	466
+3	366
+3	406
+3	190
+3	406
+3	114
+3	258
+3	90
+3	262
+3	348
+3	424
+3	12
+3	396
+3	164
+3	454
+3	478
+3	298
+3	164
+3	424
+3	382
+3	70
+3	480
+3	24
+3	104
+3	70
+3	438
+3	414
+3	200
+3	360
+3	248
+3	444
+3	120
+3	230
+3	478
+3	178
+3	468
+3	310
+3	460
+3	480
+3	136
+3	172
+3	214
+3	462
+3	406
+3	454
+3	384
+3	256
+3	26
+3	134
+3	384
+3	18
+3	462
+3	492
+3	100
+3	298
+3	498
+3	146
+3	458
+3	362
+3	186
+3	348
+3	18
+3	344
+3	84
+3	28
+3	448
+3	152
+3	348
+3	194
+3	414
+3	222
+3	126
+3	90
+3	400
+3	200
+4	238
+4	86
+4	278
+4	98
+4	484
+4	150
+4	224
+4	66
+4	128
+4	146
+4	406
+4	374
+4	152
+4	82
+4	166
+4	430
+4	252
+4	292
+4	338
+4	446
+4	394
+4	482
+4	174
+4	494
+4	466
+4	208
+4	174
+4	396
+4	162
+4	266
+4	342
+4	0
+4	128
+4	316
+4	302
+4	438
+4	170
+4	20
+4	378
+4	92
+4	72
+4	4
+4	280
+4	208
+4	356
+4	382
+4	498
+4	386
+4	192
+4	286
+4	176
+4	54
+4	138
+4	216
+4	430
+4	278
+4	176
+4	318
+4	332
+4	180
+4	284
+4	12
+4	230
+4	260
+4	404
+4	384
+4	272
+4	138
+4	84
+4	348
+4	466
+4	58
+4	8
+4	230
+4	208
+4	348
+4	24
+4	172
+4	42
+4	158
+4	496
+4	0
+4	322
+4	468
+4	454
+4	100
+4	298
+4	418
+4	96
+4	26
+4	230
+4	120
+4	404
+4	436
+4	156
+4	468
+4	308
+4	196
+4	288
+4	98
+4	282
+4	318
+4	318
+4	470
+4	316
+4	0
+4	490
+4	364
+4	118
+4	134
+4	282
+4	138
+4	238
+4	118
+4	72
+4	90
+4	10
+4	306
+4	224
+4	242
+4	392
+4	272
+4	242
+4	452
+4	226
+4	402
+4	396
+4	58
+4	336
+4	168
+4	34
+4	472
+4	322
+4	498
+4	160
+4	42
+4	430
+4	458
+4	78
+4	76
+4	492
+4	218
+4	228
+4	138
+4	30
+4	64
+4	468
+4	76
+4	74
+4	342
+4	230
+4	368
+4	296
+4	216
+4	344
+4	274
+4	116
+4	256
+4	70
+4	480
+4	288
+4	244
+4	438
+4	128
+4	432
+4	202
+4	316
+4	280
+4	2
+4	80
+4	44
+4	104
+4	466
+4	366
+4	406
+4	190
+4	406
+4	114
+4	258
+4	90
+4	262
+4	348
+4	424
+4	12
+4	396
+4	164
+4	454
+4	478
+4	298
+4	164
+4	424
+4	382
+4	70
+4	480
+4	24
+4	104
+4	70
+4	438
+4	414
+4	200
+4	360
+4	248
+4	444
+4	120
+4	230
+4	478
+4	178
+4	468
+4	310
+4	460
+4	480
+4	136
+4	172
+4	214
+4	462
+4	406
+4	454
+4	384
+4	256
+4	26
+4	134
+4	384
+4	18
+4	462
+4	492
+4	100
+4	298
+4	498
+4	146
+4	458
+4	362
+4	186
+4	348
+4	18
+4	344
+4	84
+4	28
+4	448
+4	152
+4	348
+4	194
+4	414
+4	222
+4	126
+4	90
+4	400
+4	200
+PREHOOK: query: select r1, r2
+from
+  (select
+    execute_splits(
+      "select key from srcpart where key % 2 = 0",
+      5) as (r1, r2)) t
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select r1, r2
+from
+  (select
+    execute_splits(
+      "select key from srcpart where key % 2 = 0",
+      5) as (r1, r2)) t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+1	238
+1	86
+1	278
+1	98
+1	484
+1	150
+1	224
+1	66
+1	128
+1	146
+1	406
+1	374
+1	152
+1	82
+1	166
+1	430
+1	252
+1	292
+1	338
+1	446
+1	394
+1	482
+1	174
+1	494
+1	466
+1	208
+1	174
+1	396
+1	162
+1	266
+1	342
+1	0
+1	128
+1	316
+1	302
+1	438
+1	170
+1	20
+1	378
+1	92
+1	72
+1	4
+1	280
+1	208
+1	356
+1	382
+1	498
+1	386
+1	192
+1	286
+1	176
+1	54
+1	138
+1	216
+1	430
+1	278
+1	176
+1	318
+1	332
+1	180
+1	284
+1	12
+1	230
+1	260
+1	404
+1	384
+1	272
+1	138
+1	84
+1	348
+1	466
+1	58
+1	8
+1	230
+1	208
+1	348
+1	24
+1	172
+1	42
+1	158
+1	496
+1	0
+1	322
+1	468
+1	454
+1	100
+1	298
+1	418
+1	96
+1	26
+1	230
+1	120
+1	404
+1	436
+1	156
+1	468
+1	308
+1	196
+1	288
+1	98
+1	282
+1	318
+1	318
+1	470
+1	316
+1	0
+1	490
+1	364
+1	118
+1	134
+1	282
+1	138
+1	238
+1	118
+1	72
+1	90
+1	10
+1	306
+1	224
+1	242
+1	392
+1	272
+1	242
+1	452
+1	226
+1	402
+1	396
+1	58
+1	336
+1	168
+1	34
+1	472
+1	322
+1	498
+1	160
+1	42
+1	430
+1	458
+1	78
+1	76
+1	492
+1	218
+1	228
+1	138
+1	30
+1	64
+1	468
+1	76
+1	74
+1	342
+1	230
+1	368
+1	296
+1	216
+1	344
+1	274
+1	116
+1	256
+1	70
+1	480
+1	288
+1	244
+1	438
+1	128
+1	432
+1	202
+1	316
+1	280
+1	2
+1	80
+1	44
+1	104
+1	466
+1	366
+1	406
+1	190
+1	406
+1	114
+1	258
+1	90
+1	262
+1	348
+1	424
+1	12
+1	396
+1	164
+1	454
+1	478
+1	298
+1	164
+1	424
+1	382
+1	70
+1	480
+1	24
+1	104
+1	70
+1	438
+1	414
+1	200
+1	360
+1	248
+1	444
+1	120
+1	230
+1	478
+1	178
+1	468
+1	310
+1	460
+1	480
+1	136
+1	172
+1	214
+1	462
+1	406
+1	454
+1	384
+1	256
+1	26
+1	134
+1	384
+1	18
+1	462
+1	492
+1	100
+1	298
+1	498
+1	146
+1	458
+1	362
+1	186
+1	348
+1	18
+1	344
+1	84
+1	28
+1	448
+1	152
+1	348
+1	194
+1	414
+1	222
+1	126
+1	90
+1	400
+1	200
+2	238
+2	86
+2	278
+2	98
+2	484
+2	150
+2	224
+2	66
+2	128
+2	146
+2	406
+2	374
+2	152
+2	82
+2	166
+2	430
+2	252
+2	292
+2	338
+2	446
+2	394
+2	482
+2	174
+2	494
+2	466
+2	208
+2	174
+2	396
+2	162
+2	266
+2	342
+2	0
+2	128
+2	316
+2	302
+2	438
+2	170
+2	20
+2	378
+2	92
+2	72
+2	4
+2	280
+2	208
+2	356
+2	382
+2	498
+2	386
+2	192
+2	286
+2	176
+2	54
+2	138
+2	216
+2	430
+2	278
+2	176
+2	318
+2	332
+2	180
+2	284
+2	12
+2	230
+2	260
+2	404
+2	384
+2	272
+2	138
+2	84
+2	348
+2	466
+2	58
+2	8
+2	230
+2	208
+2	348
+2	24
+2	172
+2	42
+2	158
+2	496
+2	0
+2	322
+2	468
+2	454
+2	100
+2	298
+2	418
+2	96
+2	26
+2	230
+2	120
+2	404
+2	436
+2	156
+2	468
+2	308
+2	196
+2	288
+2	98
+2	282
+2	318
+2	318
+2	470
+2	316
+2	0
+2	490
+2	364
+2	118
+2	134
+2	282
+2	138
+2	238
+2	118
+2	72
+2	90
+2	10
+2	306
+2	224
+2	242
+2	392
+2	272
+2	242
+2	452
+2	226
+2	402
+2	396
+2	58
+2	336
+2	168
+2	34
+2	472
+2	322
+2	498
+2	160
+2	42
+2	430
+2	458
+2	78
+2	76
+2	492
+2	218
+2	228
+2	138
+2	30
+2	64
+2	468
+2	76
+2	74
+2	342
+2	230
+2	368
+2	296
+2	216
+2	344
+2	274
+2	116
+2	256
+2	70
+2	480
+2	288
+2	244
+2	438
+2	128
+2	432
+2	202
+2	316
+2	280
+2	2
+2	80
+2	44
+2	104
+2	466
+2	366
+2	406
+2	190
+2	406
+2	114
+2	258
+2	90
+2	262
+2	348
+2	424
+2	12
+2	396
+2	164
+2	454
+2	478
+2	298
+2	164
+2	424
+2	382
+2	70
+2	480
+2	24
+2	104
+2	70
+2	438
+2	414
+2	200
+2	360
+2	248
+2	444
+2	120
+2	230
+2	478
+2	178
+2	468
+2	310
+2	460
+2	480
+2	136
+2	172
+2	214
+2	462
+2	406
+2	454
+2	384
+2	256
+2	26
+2	134
+2	384
+2	18
+2	462
+2	492
+2	100
+2	298
+2	498
+2	146
+2	458
+2	362
+2	186
+2	348
+2	18
+2	344
+2	84
+2	28
+2	448
+2	152
+2	348
+2	194
+2	414
+2	222
+2	126
+2	90
+2	400
+2	200
+3	238
+3	86
+3	278
+3	98
+3	484
+3	150
+3	224
+3	66
+3	128
+3	146
+3	406
+3	374
+3	152
+3	82
+3	166
+3	430
+3	252
+3	292
+3	338
+3	446
+3	394
+3	482
+3	174
+3	494
+3	466
+3	208
+3	174
+3	396
+3	162
+3	266
+3	342
+3	0
+3	128
+3	316
+3	302
+3	438
+3	170
+3	20
+3	378
+3	92
+3	72
+3	4
+3	280
+3	208
+3	356
+3	382
+3	498
+3	386
+3	192
+3	286
+3	176
+3	54
+3	138
+3	216
+3	430
+3	278
+3	176
+3	318
+3	332
+3	180
+3	284
+3	12
+3	230
+3	260
+3	404
+3	384
+3	272
+3	138
+3	84
+3	348
+3	466
+3	58
+3	8
+3	230
+3	208
+3	348
+3	24
+3	172
+3	42
+3	158
+3	496
+3	0
+3	322
+3	468
+3	454
+3	100
+3	298
+3	418
+3	96
+3	26
+3	230
+3	120
+3	404
+3	436
+3	156
+3	468
+3	308
+3	196
+3	288
+3	98
+3	282
+3	318
+3	318
+3	470
+3	316
+3	0
+3	490
+3	364
+3	118
+3	134
+3	282
+3	138
+3	238
+3	118
+3	72
+3	90
+3	10
+3	306
+3	224
+3	242
+3	392
+3	272
+3	242
+3	452
+3	226
+3	402
+3	396
+3	58
+3	336
+3	168
+3	34
+3	472
+3	322
+3	498
+3	160
+3	42
+3	430
+3	458
+3	78
+3	76
+3	492
+3	218
+3	228
+3	138
+3	30
+3	64
+3	468
+3	76
+3	74
+3	342
+3	230
+3	368
+3	296
+3	216
+3	344
+3	274
+3	116
+3	256
+3	70
+3	480
+3	288
+3	244
+3	438
+3	128
+3	432
+3	202
+3	316
+3	280
+3	2
+3	80
+3	44
+3	104
+3	466
+3	366
+3	406
+3	190
+3	406
+3	114
+3	258
+3	90
+3	262
+3	348
+3	424
+3	12
+3	396
+3	164
+3	454
+3	478
+3	298
+3	164
+3	424
+3	382
+3	70
+3	480
+3	24
+3	104
+3	70
+3	438
+3	414
+3	200
+3	360
+3	248
+3	444
+3	120
+3	230
+3	478
+3	178
+3	468
+3	310
+3	460
+3	480
+3	136
+3	172
+3	214
+3	462
+3	406
+3	454
+3	384
+3	256
+3	26
+3	134
+3	384
+3	18
+3	462
+3	492
+3	100
+3	298
+3	498
+3	146
+3	458
+3	362
+3	186
+3	348
+3	18
+3	344
+3	84
+3	28
+3	448
+3	152
+3	348
+3	194
+3	414
+3	222
+3	126
+3	90
+3	400
+3	200
+4	238
+4	86
+4	278
+4	98
+4	484
+4	150
+4	224
+4	66
+4	128
+4	146
+4	406
+4	374
+4	152
+4	82
+4	166
+4	430
+4	252
+4	292
+4	338
+4	446
+4	394
+4	482
+4	174
+4	494
+4	466
+4	208
+4	174
+4	396
+4	162
+4	266
+4	342
+4	0
+4	128
+4	316
+4	302
+4	438
+4	170
+4	20
+4	378
+4	92
+4	72
+4	4
+4	280
+4	208
+4	356
+4	382
+4	498
+4	386
+4	192
+4	286
+4	176
+4	54
+4	138
+4	216
+4	430
+4	278
+4	176
+4	318
+4	332
+4	180
+4	284
+4	12
+4	230
+4	260
+4	404
+4	384
+4	272
+4	138
+4	84
+4	348
+4	466
+4	58
+4	8
+4	230
+4	208
+4	348
+4	24
+4	172
+4	42
+4	158
+4	496
+4	0
+4	322
+4	468
+4	454
+4	100
+4	298
+4	418
+4	96
+4	26
+4	230
+4	120
+4	404
+4	436
+4	156
+4	468
+4	308
+4	196
+4	288
+4	98
+4	282
+4	318
+4	318
+4	470
+4	316
+4	0
+4	490
+4	364
+4	118
+4	134
+4	282
+4	138
+4	238
+4	118
+4	72
+4	90
+4	10
+4	306
+4	224
+4	242
+4	392
+4	272
+4	242
+4	452
+4	226
+4	402
+4	396
+4	58
+4	336
+4	168
+4	34
+4	472
+4	322
+4	498
+4	160
+4	42
+4	430
+4	458
+4	78
+4	76
+4	492
+4	218
+4	228
+4	138
+4	30
+4	64
+4	468
+4	76
+4	74
+4	342
+4	230
+4	368
+4	296
+4	216
+4	344
+4	274
+4	116
+4	256
+4	70
+4	480
+4	288
+4	244
+4	438
+4	128
+4	432
+4	202
+4	316
+4	280
+4	2
+4	80
+4	44
+4	104
+4	466
+4	366
+4	406
+4	190
+4	406
+4	114
+4	258
+4	90
+4	262
+4	348
+4	424
+4	12
+4	396
+4	164
+4	454
+4	478
+4	298
+4	164
+4	424
+4	382
+4	70
+4	480
+4	24
+4	104
+4	70
+4	438
+4	414
+4	200
+4	360
+4	248
+4	444
+4	120
+4	230
+4	478
+4	178
+4	468
+4	310
+4	460
+4	480
+4	136
+4	172
+4	214
+4	462
+4	406
+4	454
+4	384
+4	256
+4	26
+4	134
+4	384
+4	18
+4	462
+4	492
+4	100
+4	298
+4	498
+4	146
+4	458
+4	362
+4	186
+4	348
+4	18
+4	344
+4	84
+4	28
+4	448
+4	152
+4	348
+4	194
+4	414
+4	222
+4	126
+4	90
+4	400
+4	200
+PREHOOK: query: select count(*) from (select key from srcpart where key % 2 = 0) t
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from (select key from srcpart where key % 2 = 0) t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+988


[10/39] hive git commit: HIVE-13304: Merge master into llap branch

Posted by jd...@apache.org.
HIVE-13304: Merge master into llap branch


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2945c3b2
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2945c3b2
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2945c3b2

Branch: refs/heads/master
Commit: 2945c3b2d04304326b48bdf0e646fb36be148fe5
Parents: 81b26df a17122f
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu Mar 17 15:40:06 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu Mar 17 15:40:06 2016 -0700

----------------------------------------------------------------------
 beeline/pom.xml                                 |    22 +
 .../org/apache/hadoop/hive/cli/CliDriver.java   |     7 +-
 common/pom.xml                                  |    19 +
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    88 +-
 .../hive/common/metrics/MetricsTestUtils.java   |     9 +
 .../metrics/metrics2/TestCodahaleMetrics.java   |    26 +-
 .../apache/hadoop/hive/conf/TestHiveConf.java   |    10 +
 data/files/dec_old.avro                         |   Bin 0 -> 331 bytes
 data/files/over4_null                           |     5 +
 .../hive/hcatalog/cli/TestSemanticAnalysis.java |    15 +
 .../org/apache/hive/minikdc/MiniHiveKdc.java    |    15 +
 .../hive/minikdc/TestJdbcWithDBTokenStore.java  |    40 +
 .../hive/minikdc/TestJdbcWithMiniKdc.java       |    12 +-
 .../hive/thrift/TestHadoopAuthBridge23.java     |    63 +-
 .../hive/metastore/TestHiveMetaStore.java       |     1 +
 .../hadoop/hive/thrift/TestDBTokenStore.java    |     3 +-
 .../hive/thrift/TestZooKeeperTokenStore.java    |    12 +-
 .../test/java/org/apache/hive/jdbc/TestSSL.java |     2 +
 .../cli/TestEmbeddedThriftBinaryCLIService.java |     1 +
 .../TestOperationLoggingAPIWithTez.java         |     2 +-
 .../service/cli/session/TestQueryDisplay.java   |   180 +
 itests/qtest/pom.xml                            |     2 +-
 .../test/resources/testconfiguration.properties |     2 +
 .../org/apache/hive/jdbc/HiveConnection.java    |    30 +-
 .../hive/jdbc/HttpTokenAuthInterceptor.java     |    47 +
 jdbc/src/java/org/apache/hive/jdbc/Utils.java   |     4 +
 .../hive/llap/counters/LlapIOCounters.java      |    37 +
 .../llap/IncrementalObjectSizeEstimator.java    |     7 +-
 .../apache/hadoop/hive/llap/cache/Cache.java    |    27 -
 .../hadoop/hive/llap/cache/LowLevelCache.java   |     4 +-
 .../hive/llap/cache/LowLevelCacheImpl.java      |    28 +-
 .../hadoop/hive/llap/cache/NoopCache.java       |    33 -
 .../hive/llap/cli/LlapOptionsProcessor.java     |    46 +-
 .../hadoop/hive/llap/cli/LlapServiceDriver.java |    20 +
 .../hive/llap/counters/FragmentCountersMap.java |    46 +
 .../llap/counters/QueryFragmentCounters.java    |    65 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |     6 +-
 .../hive/llap/daemon/impl/LlapTaskReporter.java |    14 +-
 .../llap/daemon/impl/TaskRunnerCallable.java    |    14 +-
 .../hive/llap/io/api/impl/LlapInputFormat.java  |    30 +-
 .../hive/llap/io/api/impl/LlapIoImpl.java       |     6 +-
 .../llap/io/decode/EncodedDataConsumer.java     |    77 +-
 .../llap/io/decode/OrcColumnVectorProducer.java |     8 +-
 .../llap/io/decode/OrcEncodedDataConsumer.java  |    11 +-
 .../llap/io/encoded/OrcEncodedDataReader.java   |   214 +-
 .../hive/llap/io/metadata/OrcFileMetadata.java  |    33 +-
 .../hive/llap/io/metadata/OrcMetadataCache.java |    12 +-
 .../llap/io/metadata/OrcStripeMetadata.java     |    19 +-
 .../hive/llap/security/SecretManager.java       |    39 +-
 llap-server/src/main/resources/package.py       |    11 +
 llap-server/src/main/resources/templates.py     |     6 +-
 metastore/if/hive_metastore.thrift              |    28 +-
 .../apache/hadoop/hive/metastore/Metastore.java |    22 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 22340 ++++++++++-------
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  1409 +-
 .../ThriftHiveMetastore_server.skeleton.cpp     |    40 +
 .../hive/metastore/api/ThriftHiveMetastore.java |  9220 ++++++-
 .../gen-php/metastore/ThriftHiveMetastore.php   |  1756 ++
 .../hive_metastore/ThriftHiveMetastore-remote   |    56 +
 .../hive_metastore/ThriftHiveMetastore.py       |  1639 +-
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   450 +
 .../hive/metastore/AcidEventListener.java       |    94 +
 .../hadoop/hive/metastore/FileFormatProxy.java  |     6 +-
 .../hive/metastore/FileMetadataHandler.java     |     2 +-
 .../hadoop/hive/metastore/HiveAlterHandler.java |     2 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   265 +-
 .../hive/metastore/HiveMetaStoreClient.java     |    85 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |    23 +-
 .../hive/metastore/MetaStoreDirectSql.java      |     8 +-
 .../hadoop/hive/metastore/ObjectStore.java      |    14 +-
 .../hive/metastore/TSetIpAddressProcessor.java  |     2 +-
 .../TransactionalValidationListener.java        |    11 +
 .../filemeta/OrcFileMetadataHandler.java        |    15 +-
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |    39 +-
 .../metastore/txn/CompactionTxnHandler.java     |     2 +-
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    |    20 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |   810 +-
 .../hadoop/hive/metastore/txn/TxnStore.java     |    37 +-
 .../hadoop/hive/metastore/txn/TxnUtils.java     |    18 +
 .../hadoop/hive/metastore/model/MOrder.java     |     4 +-
 .../hive/metastore/IpAddressListener.java       |     2 +-
 .../hive/metastore/hbase/TestHBaseStore.java    |    24 +-
 .../hbase/TestHBaseStoreBitVector.java          |    34 +-
 .../hbase/TestSharedStorageDescriptor.java      |    11 +-
 .../metastore/txn/TestTxnHandlerNegative.java   |    10 +-
 .../protobuf-java/org/apache/orc/OrcProto.java  |    16 +
 orc/src/java/org/apache/orc/FileMetadata.java   |     2 +-
 orc/src/java/org/apache/orc/OrcFile.java        |     5 +-
 orc/src/java/org/apache/orc/impl/InStream.java  |     2 +-
 .../java/org/apache/orc/impl/MemoryManager.java |     8 +-
 orc/src/java/org/apache/orc/impl/OutStream.java |     2 +-
 orc/src/protobuf/orc_proto.proto                |     2 +
 .../test/org/apache/orc/impl/TestOutStream.java |    43 +
 ql/pom.xml                                      |    22 +
 .../java/org/apache/hadoop/hive/ql/Driver.java  |    93 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |     1 +
 .../org/apache/hadoop/hive/ql/QueryDisplay.java |   133 +-
 .../org/apache/hadoop/hive/ql/QueryPlan.java    |    14 +
 .../hive/ql/exec/AbstractFileMergeOperator.java |    23 +-
 .../hadoop/hive/ql/exec/ColumnStatsTask.java    |    17 +-
 .../hive/ql/exec/ColumnStatsUpdateTask.java     |    12 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    89 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |     4 +
 .../hadoop/hive/ql/exec/MapJoinOperator.java    |     2 +
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |     9 +-
 .../hive/ql/exec/OrcFileMergeOperator.java      |    14 +-
 .../hadoop/hive/ql/exec/ReduceSinkOperator.java |     1 +
 .../hadoop/hive/ql/exec/StatsNoJobTask.java     |    41 +-
 .../apache/hadoop/hive/ql/exec/StatsTask.java   |    14 +-
 .../hadoop/hive/ql/exec/TableScanOperator.java  |    10 +
 .../org/apache/hadoop/hive/ql/exec/Task.java    |    87 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |     4 +-
 .../hive/ql/exec/errors/TaskLogProcessor.java   |     2 +-
 .../hadoop/hive/ql/exec/mr/ExecDriver.java      |     8 +-
 .../hadoop/hive/ql/exec/mr/MapRedTask.java      |     8 +-
 .../persistence/HybridHashTableContainer.java   |    22 +-
 .../persistence/MapJoinBytesTableContainer.java |    36 +-
 .../hive/ql/exec/persistence/MapJoinKey.java    |    12 +-
 .../hadoop/hive/ql/exec/tez/DagUtils.java       |     2 +-
 .../ql/exec/tez/DynamicPartitionPruner.java     |     2 +-
 .../hadoop/hive/ql/exec/tez/TezJobMonitor.java  |   238 +-
 .../ql/exec/vector/VectorizationContext.java    |     5 +-
 ...AbstractFilterStringColLikeStringScalar.java |   168 +-
 .../FilterStringColLikeStringScalar.java        |    18 +
 .../VectorReduceSinkCommonOperator.java         |    66 +-
 .../hadoop/hive/ql/history/HiveHistory.java     |     2 +-
 .../ql/hooks/UpdateInputAccessTimeHook.java     |    20 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |    47 +-
 .../org/apache/hadoop/hive/ql/io/HdfsUtils.java |    23 +-
 .../hadoop/hive/ql/io/SyntheticFileId.java      |   100 +
 .../hadoop/hive/ql/io/orc/ExternalCache.java    |   338 +
 .../hadoop/hive/ql/io/orc/LocalCache.java       |   112 +
 .../io/orc/MetastoreExternalCachesByConf.java   |    82 +
 .../hive/ql/io/orc/OrcFileFormatProxy.java      |    14 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   698 +-
 .../hive/ql/io/orc/OrcNewInputFormat.java       |    16 +-
 .../hadoop/hive/ql/io/orc/OrcOutputFormat.java  |    17 +-
 .../apache/hadoop/hive/ql/io/orc/OrcSplit.java  |    46 +-
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |   125 +-
 .../ql/io/orc/encoded/EncodedReaderImpl.java    |    32 +-
 .../hive/ql/io/orc/encoded/OrcBatchKey.java     |    20 +-
 .../hive/ql/io/orc/encoded/OrcCacheKey.java     |    58 -
 .../hadoop/hive/ql/io/orc/encoded/Reader.java   |    10 +-
 .../hive/ql/io/orc/encoded/ReaderImpl.java      |     4 +-
 .../hive/ql/io/orc/encoded/StreamUtils.java     |     1 -
 .../serde/ArrayWritableObjectInspector.java     |     4 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |     2 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   111 +-
 .../hive/ql/metadata/PartitionIterable.java     |     2 +-
 .../BucketingSortingReduceSinkOptimizer.java    |    51 +-
 .../hadoop/hive/ql/optimizer/ColumnPruner.java  |     4 +
 .../ql/optimizer/ColumnPrunerProcFactory.java   |    12 +
 .../optimizer/ConstantPropagateProcFactory.java |    32 +-
 .../hive/ql/optimizer/ConvertJoinMapJoin.java   |    85 +-
 .../ql/optimizer/ReduceSinkMapJoinProc.java     |    15 +-
 .../optimizer/SortedDynPartitionOptimizer.java  |    78 +-
 .../ql/optimizer/calcite/RelOptHiveTable.java   |    10 +-
 .../calcite/reloperators/HiveTableScan.java     |    23 +-
 .../calcite/rules/HiveRelFieldTrimmer.java      |    42 +-
 .../calcite/translator/ASTBuilder.java          |     8 +
 .../calcite/translator/ASTConverter.java        |    49 +-
 .../calcite/translator/ExprNodeConverter.java   |    12 +
 .../calcite/translator/HiveOpConverter.java     |    39 +-
 .../correlation/ReduceSinkDeDuplication.java    |    34 +-
 .../ql/optimizer/index/RewriteGBUsingIndex.java |     1 +
 .../RewriteQueryUsingAggregateIndexCtx.java     |     1 +
 .../physical/BucketingSortingOpProcFactory.java |     3 +-
 .../hive/ql/optimizer/physical/LlapDecider.java |    11 +-
 .../physical/MetadataOnlyOptimizer.java         |     3 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |    13 +-
 .../spark/SparkReduceSinkMapJoinProc.java       |    23 +-
 .../stats/annotation/StatsRulesProcFactory.java |   184 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |    27 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |    75 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |     2 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |    48 +-
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |     4 +-
 .../hadoop/hive/ql/parse/IndexUpdater.java      |     1 +
 .../hadoop/hive/ql/parse/PTFInvocationSpec.java |    25 +-
 .../hadoop/hive/ql/parse/PTFTranslator.java     |    26 +-
 .../hadoop/hive/ql/parse/ParseContext.java      |    35 +-
 .../org/apache/hadoop/hive/ql/parse/QB.java     |    30 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   156 +-
 .../hadoop/hive/ql/parse/TaskCompiler.java      |     3 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |     2 +
 .../hadoop/hive/ql/parse/WindowingSpec.java     |    56 +-
 .../HiveAuthorizationTaskFactoryImpl.java       |     1 +
 .../hadoop/hive/ql/plan/CreateTableDesc.java    |    26 +
 .../hadoop/hive/ql/plan/PTFDeserializer.java    |     5 +-
 .../apache/hadoop/hive/ql/plan/PlanUtils.java   |    35 +-
 .../hadoop/hive/ql/plan/ReduceSinkDesc.java     |    23 +-
 .../hadoop/hive/ql/plan/TableScanDesc.java      |    16 +
 .../hive/ql/plan/ptf/OrderExpressionDef.java    |    13 +-
 .../hive/ql/plan/ptf/PTFExpressionDef.java      |     3 +-
 .../plan/ptf/PartitionedTableFunctionDef.java   |    12 +-
 .../hive/ql/plan/ptf/ValueBoundaryDef.java      |    16 +-
 .../HiveAuthorizationProviderBase.java          |    16 +-
 .../authorization/plugin/HiveV1Authorizer.java  |    21 +-
 .../hadoop/hive/ql/session/SessionState.java    |    28 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |     2 +
 .../hive/ql/txn/compactor/CompactorMR.java      |     2 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java |     2 +-
 .../org/apache/hadoop/hive/ql/udf/UDFChr.java   |   101 +
 .../apache/hadoop/hive/ql/udf/UDFReplace.java   |    50 +
 .../ql/udf/generic/GenericUDFFormatNumber.java  |    77 +-
 .../hive/ql/udf/ptf/WindowingTableFunction.java |   302 +-
 ql/src/test/data/rc-file-v0.rc                  |   Bin 216 -> 0 bytes
 .../apache/hadoop/hive/ql/TestTxnCommands2.java |    22 +-
 .../ql/exec/errors/TestTaskLogProcessor.java    |     8 +-
 .../TestVectorStringExpressions.java            |   156 +-
 .../apache/hadoop/hive/ql/hooks/TestHooks.java  |     2 +-
 .../apache/hadoop/hive/ql/io/TestRCFile.java    |     3 +-
 .../hive/ql/io/orc/TestInputOutputFormat.java   |    61 +-
 .../hadoop/hive/ql/io/orc/TestOrcFile.java      |   113 +
 .../hive/ql/io/orc/TestOrcSplitElimination.java |   405 +-
 .../hadoop/hive/ql/io/orc/TestReaderImpl.java   |   151 +
 .../hive/ql/lockmgr/TestDbTxnManager2.java      |   209 +-
 .../hive/ql/udf/generic/TestGenericUDFChr.java  |   156 +
 .../ql/udf/generic/TestGenericUDFReplace.java   |    56 +
 .../clientnegative/alter_external_acid.q        |     9 +
 .../clientnegative/authorization_view_1.q       |    13 +
 .../clientnegative/authorization_view_2.q       |    17 +
 .../clientnegative/authorization_view_3.q       |    15 +
 .../clientnegative/authorization_view_4.q       |    23 +
 .../clientnegative/authorization_view_5.q       |    16 +
 .../clientnegative/authorization_view_6.q       |    18 +
 .../clientnegative/authorization_view_7.q       |    18 +
 .../authorization_view_disable_cbo_1.q          |    14 +
 .../authorization_view_disable_cbo_2.q          |    17 +
 .../authorization_view_disable_cbo_3.q          |    16 +
 .../authorization_view_disable_cbo_4.q          |    24 +
 .../authorization_view_disable_cbo_5.q          |    17 +
 .../authorization_view_disable_cbo_6.q          |    19 +
 .../authorization_view_disable_cbo_7.q          |    19 +
 .../test/queries/clientnegative/avro_decimal.q  |    17 +
 .../clientnegative/compact_non_acid_table.q     |    11 +
 .../clientnegative/create_external_acid.q       |     6 +
 .../clientnegative/udf_format_number_wrong6.q   |     2 -
 .../test/queries/clientpositive/add_jar_pfile.q |     2 +-
 .../clientpositive/annotate_stats_filter.q      |    12 +-
 .../clientpositive/authorization_view_1.q       |    59 +
 .../clientpositive/authorization_view_2.q       |    16 +
 .../clientpositive/authorization_view_3.q       |    18 +
 .../clientpositive/authorization_view_4.q       |    18 +
 .../authorization_view_disable_cbo_1.q          |    70 +
 .../authorization_view_disable_cbo_2.q          |    17 +
 .../authorization_view_disable_cbo_3.q          |    19 +
 .../authorization_view_disable_cbo_4.q          |    19 +
 .../queries/clientpositive/avro_decimal_old.q   |    14 +
 ql/src/test/queries/clientpositive/constprog2.q |    17 -
 .../clientpositive/constprog_partitioner.q      |    25 -
 .../queries/clientpositive/cross_join_merge.q   |    17 +
 .../queries/clientpositive/dbtxnmgr_compact1.q  |     2 +-
 .../queries/clientpositive/dbtxnmgr_compact2.q  |     2 +-
 .../queries/clientpositive/dbtxnmgr_compact3.q  |     2 +-
 .../clientpositive/encryption_drop_partition.q  |    10 +
 .../clientpositive/encryption_drop_table.q      |     9 +-
 ql/src/test/queries/clientpositive/keyword_3.q  |     8 +
 ql/src/test/queries/clientpositive/orc_create.q |    12 +
 .../clientpositive/orc_merge_incompat3.q        |    14 +
 ql/src/test/queries/clientpositive/order_null.q |    29 +
 .../reduceSinkDeDuplication_pRS_key_empty.q     |    60 +
 .../test/queries/clientpositive/skiphf_aggr.q   |    42 +
 ql/src/test/queries/clientpositive/udf_chr.q    |    25 +
 .../queries/clientpositive/udf_format_number.q  |    36 +-
 .../test/queries/clientpositive/udf_replace.q   |     9 +
 .../test/queries/clientpositive/vector_udf1.q   |   327 +
 .../clientpositive/windowing_order_null.q       |    35 +
 .../clientpositive/windowing_range_multiorder.q |    34 +
 .../resources/orc-file-dump-bloomfilter.out     |     2 +-
 .../resources/orc-file-dump-bloomfilter2.out    |     2 +-
 .../orc-file-dump-dictionary-threshold.out      |     2 +-
 ql/src/test/resources/orc-file-dump.json        |     2 +-
 ql/src/test/resources/orc-file-dump.out         |     2 +-
 ql/src/test/resources/orc-file-has-null.out     |     2 +-
 ql/src/test/resources/rc-file-v0.rc             |   Bin 0 -> 216 bytes
 .../clientnegative/alter_external_acid.q.out    |    13 +
 .../clientnegative/authorization_part.q.out     |     2 +
 .../clientnegative/authorization_view_1.q.out   |    33 +
 .../clientnegative/authorization_view_2.q.out   |    39 +
 .../clientnegative/authorization_view_3.q.out   |    39 +
 .../clientnegative/authorization_view_4.q.out   |    71 +
 .../clientnegative/authorization_view_5.q.out   |    35 +
 .../clientnegative/authorization_view_6.q.out   |    45 +
 .../clientnegative/authorization_view_7.q.out   |    45 +
 .../authorization_view_disable_cbo_1.q.out      |    33 +
 .../authorization_view_disable_cbo_2.q.out      |    39 +
 .../authorization_view_disable_cbo_3.q.out      |    39 +
 .../authorization_view_disable_cbo_4.q.out      |    71 +
 .../authorization_view_disable_cbo_5.q.out      |    35 +
 .../authorization_view_disable_cbo_6.q.out      |    45 +
 .../authorization_view_disable_cbo_7.q.out      |    45 +
 .../results/clientnegative/avro_decimal.q.out   |    22 +
 .../clientnegative/compact_non_acid_table.q.out |    11 +
 .../clientnegative/create_external_acid.q.out   |     5 +
 .../udf_format_number_wrong1.q.out              |     2 +-
 .../udf_format_number_wrong2.q.out              |     2 +-
 .../udf_format_number_wrong4.q.out              |     2 +-
 .../udf_format_number_wrong6.q.out              |     1 -
 .../clientnegative/updateBasicStats.q.out       |     2 +
 .../clientpositive/allcolref_in_udf.q.out       |     1 +
 .../alter_partition_coltype.q.out               |     3 +
 .../alter_rename_partition_authorization.q.out  |     2 +
 .../alter_table_update_status.q.out             |     2 +
 .../clientpositive/annotate_stats_filter.q.out  |   176 +-
 .../annotate_stats_join_pkfk.q.out              |    62 +-
 .../clientpositive/annotate_stats_part.q.out    |    12 +-
 .../clientpositive/annotate_stats_table.q.out   |     1 +
 .../clientpositive/authorization_1.q.out        |     2 +
 .../clientpositive/authorization_2.q.out        |     2 +
 .../clientpositive/authorization_3.q.out        |     2 +
 .../clientpositive/authorization_4.q.out        |     2 +
 .../clientpositive/authorization_6.q.out        |     2 +
 .../authorization_create_temp_table.q.out       |     2 +
 .../clientpositive/authorization_view_1.q.out   |   263 +
 .../clientpositive/authorization_view_2.q.out   |    66 +
 .../clientpositive/authorization_view_3.q.out   |    62 +
 .../clientpositive/authorization_view_4.q.out   |    64 +
 .../authorization_view_disable_cbo_1.q.out      |   311 +
 .../authorization_view_disable_cbo_2.q.out      |    66 +
 .../authorization_view_disable_cbo_3.q.out      |    62 +
 .../authorization_view_disable_cbo_4.q.out      |    64 +
 .../auto_join_reordering_values.q.out           |     8 +
 .../clientpositive/auto_join_stats.q.out        |    58 +-
 .../clientpositive/auto_join_stats2.q.out       |    38 +-
 .../clientpositive/auto_sortmerge_join_1.q.out  |     5 +
 .../clientpositive/auto_sortmerge_join_11.q.out |     4 +
 .../clientpositive/auto_sortmerge_join_12.q.out |     1 +
 .../clientpositive/auto_sortmerge_join_2.q.out  |     4 +
 .../clientpositive/auto_sortmerge_join_3.q.out  |     5 +
 .../clientpositive/auto_sortmerge_join_4.q.out  |     5 +
 .../clientpositive/auto_sortmerge_join_5.q.out  |     5 +
 .../clientpositive/auto_sortmerge_join_7.q.out  |     5 +
 .../clientpositive/auto_sortmerge_join_8.q.out  |     5 +
 .../clientpositive/autogen_colalias.q.out       |    22 +
 .../clientpositive/avro_decimal_old.q.out       |    60 +
 .../test/results/clientpositive/bucket1.q.out   |     1 +
 .../test/results/clientpositive/bucket2.q.out   |     1 +
 .../test/results/clientpositive/bucket3.q.out   |     1 +
 .../test/results/clientpositive/bucket4.q.out   |     1 +
 .../test/results/clientpositive/bucket5.q.out   |     2 +
 .../results/clientpositive/bucket_many.q.out    |     1 +
 .../clientpositive/bucket_map_join_1.q.out      |     1 +
 .../clientpositive/bucket_map_join_2.q.out      |     1 +
 .../clientpositive/bucketcontext_1.q.out        |     2 +
 .../clientpositive/bucketcontext_2.q.out        |     2 +
 .../clientpositive/bucketcontext_3.q.out        |     2 +
 .../clientpositive/bucketcontext_4.q.out        |     2 +
 .../clientpositive/bucketcontext_5.q.out        |     2 +
 .../clientpositive/bucketcontext_6.q.out        |     2 +
 .../clientpositive/bucketcontext_7.q.out        |     2 +
 .../clientpositive/bucketcontext_8.q.out        |     2 +
 .../bucketizedhiveinputformat.q.out             |     3 +-
 .../clientpositive/bucketmapjoin10.q.out        |     1 +
 .../clientpositive/bucketmapjoin11.q.out        |     2 +
 .../clientpositive/bucketmapjoin12.q.out        |     2 +
 .../clientpositive/bucketmapjoin13.q.out        |     4 +
 .../results/clientpositive/bucketmapjoin7.q.out |    19 +-
 .../results/clientpositive/bucketmapjoin8.q.out |     2 +
 .../results/clientpositive/bucketmapjoin9.q.out |     2 +
 .../cbo_SortUnionTransposeRule.q.out            |     2 +
 .../test/results/clientpositive/cbo_const.q.out |    26 +-
 .../cbo_rp_cross_product_check_2.q.out          |     4 +
 .../cbo_rp_outer_join_ppr.q.java1.7.out         |     4 +
 .../clientpositive/cbo_rp_unionDistinct_2.q.out |     6 +
 .../clientpositive/char_nested_types.q.out      |     2 +
 .../test/results/clientpositive/cluster.q.out   |   180 +-
 .../clientpositive/colstats_all_nulls.q.out     |     3 +
 .../clientpositive/column_access_stats.q.out    |    28 +-
 .../clientpositive/columnstats_partlvl.q.out    |     2 +
 .../clientpositive/columnstats_tbllvl.q.out     |     2 +
 .../test/results/clientpositive/combine2.q.out  |     1 +
 .../constantPropagateForSubQuery.q.out          |     2 +
 .../clientpositive/correlationoptimizer12.q.out |     4 +-
 .../clientpositive/create_default_prop.q.out    |     1 +
 .../results/clientpositive/create_view.q.out    |     2 +-
 .../clientpositive/cross_join_merge.q.out       |   490 +
 .../clientpositive/cross_product_check_1.q.out  |     4 +
 .../clientpositive/cross_product_check_2.q.out  |     4 +
 ql/src/test/results/clientpositive/ctas.q.out   |    24 +-
 .../test/results/clientpositive/ctas_char.q.out |     2 +
 .../results/clientpositive/ctas_colname.q.out   |    25 +-
 .../test/results/clientpositive/ctas_date.q.out |     4 +
 .../ctas_uses_database_location.q.out           |     2 +
 .../results/clientpositive/ctas_varchar.q.out   |     2 +
 ql/src/test/results/clientpositive/cte_2.q.out  |     1 +
 ql/src/test/results/clientpositive/cte_4.q.out  |     1 +
 .../test/results/clientpositive/database.q.out  |     2 +
 .../clientpositive/dbtxnmgr_compact1.q.out      |     4 +-
 .../clientpositive/dbtxnmgr_compact2.q.out      |     4 +-
 .../clientpositive/dbtxnmgr_compact3.q.out      |     4 +-
 .../results/clientpositive/dbtxnmgr_ddl1.q.out  |     2 +
 .../test/results/clientpositive/decimal_6.q.out |     2 +
 .../results/clientpositive/decimal_join2.q.out  |     2 +
 .../results/clientpositive/decimal_serde.q.out  |     6 +
 .../disable_merge_for_bucketing.q.out           |     1 +
 .../display_colstats_tbllvl.q.out               |     1 +
 .../dynpart_sort_opt_vectorization.q.out        |     4 +-
 .../results/clientpositive/empty_join.q.out     |     2 +
 .../encrypted/encryption_drop_partition.q.out   |    76 +
 .../encrypted/encryption_drop_table.q.out       |    27 +
 .../encryption_join_unencrypted_tbl.q.out       |     2 +
 ...on_join_with_different_encryption_keys.q.out |     2 +
 .../results/clientpositive/explain_ddl.q.out    |     2 +
 .../clientpositive/explain_logical.q.out        |     9 +-
 .../clientpositive/filter_join_breaktask.q.out  |     4 +
 .../clientpositive/fouter_join_ppr.q.out        |     8 +
 .../results/clientpositive/global_limit.q.out   |     1 +
 .../clientpositive/groupby_duplicate_key.q.out  |     3 +
 .../clientpositive/groupby_grouping_sets6.q.out |    44 +-
 .../groupby_grouping_window.q.out               |     2 +-
 .../clientpositive/groupby_map_ppr.q.out        |     1 +
 .../groupby_map_ppr_multi_distinct.q.out        |     1 +
 .../results/clientpositive/groupby_ppr.q.out    |     1 +
 .../groupby_ppr_multi_distinct.q.out            |     1 +
 .../clientpositive/groupby_resolution.q.out     |     2 +-
 .../clientpositive/groupby_sort_1_23.q.out      |    51 +-
 .../results/clientpositive/groupby_sort_6.q.out |     3 +
 .../clientpositive/groupby_sort_skew_1_23.q.out |    58 +-
 .../clientpositive/index_auto_empty.q.out       |     2 +-
 .../clientpositive/index_auto_file_format.q.out |     4 +-
 .../clientpositive/index_auto_multiple.q.out    |     2 +-
 .../clientpositive/index_auto_partitioned.q.out |     2 +-
 .../clientpositive/index_auto_update.q.out      |     2 +-
 .../index_bitmap_auto_partitioned.q.out         |     2 +-
 .../results/clientpositive/index_stale.q.out    |     2 +-
 .../index_stale_partitioned.q.out               |     2 +-
 .../clientpositive/infer_const_type.q.out       |    20 +-
 .../test/results/clientpositive/input23.q.out   |     2 +
 .../test/results/clientpositive/input46.q.out   |     4 +
 .../results/clientpositive/input_part4.q.out    |     2 +-
 .../results/clientpositive/input_part6.q.out    |     2 +-
 .../results/clientpositive/input_part7.q.out    |    38 +-
 .../test/results/clientpositive/insert0.q.out   |     2 +
 ql/src/test/results/clientpositive/join17.q.out |     2 +
 ql/src/test/results/clientpositive/join35.q.out |     2 +
 ql/src/test/results/clientpositive/join38.q.out |     8 +-
 ql/src/test/results/clientpositive/join41.q.out |     2 +
 ql/src/test/results/clientpositive/join42.q.out |    53 +-
 ql/src/test/results/clientpositive/join9.q.out  |     2 +
 .../clientpositive/join_filters_overlap.q.out   |    19 +
 .../test/results/clientpositive/keyword_3.q.out |    32 +
 .../clientpositive/lateral_view_outer.q.out     |     2 +
 .../test/results/clientpositive/lineage2.q.out  |     2 +-
 .../test/results/clientpositive/lineage3.q.out  |    10 +-
 .../list_bucket_query_multiskew_3.q.out         |     1 +
 .../list_bucket_query_oneskew_2.q.out           |     2 +
 .../llap/bucket_map_join_tez1.q.out             |   193 +-
 .../llap/bucket_map_join_tez2.q.out             |     4 +-
 .../results/clientpositive/llap/cte_2.q.out     |     1 +
 .../results/clientpositive/llap/cte_4.q.out     |     1 +
 .../llap/dynamic_partition_pruning.q.out        |   276 +-
 .../llap/dynamic_partition_pruning_2.q.out      |     8 +-
 .../llap/hybridgrace_hashjoin_1.q.out           |    24 +-
 .../llap/hybridgrace_hashjoin_2.q.out           |    24 +-
 .../clientpositive/llap/llap_nullscan.q.out     |     8 +-
 .../results/clientpositive/llap/llap_udf.q.out  |     2 +
 .../clientpositive/llap/llapdecider.q.out       |     4 +-
 .../clientpositive/llap/mapjoin_decimal.q.out   |     2 +-
 .../test/results/clientpositive/llap/mrr.q.out  |    10 +-
 .../llap/tez_bmj_schema_evolution.q.out         |     2 +-
 .../results/clientpositive/llap/tez_dml.q.out   |     6 +-
 .../llap/tez_dynpart_hashjoin_1.q.out           |    12 +-
 .../llap/tez_dynpart_hashjoin_2.q.out           |     6 +-
 .../clientpositive/llap/tez_join_hash.q.out     |     2 +-
 .../llap/tez_join_result_complex.q.out          |    34 +
 .../clientpositive/llap/tez_join_tests.q.out    |     2 +-
 .../clientpositive/llap/tez_joins_explain.q.out |     2 +-
 .../results/clientpositive/llap/tez_smb_1.q.out |     8 +-
 .../clientpositive/llap/tez_smb_main.q.out      |    20 +-
 .../results/clientpositive/llap/tez_union.q.out |    15 +-
 .../clientpositive/llap/tez_union2.q.out        |     4 +-
 .../llap/tez_union_multiinsert.q.out            |    28 +-
 .../llap/tez_vector_dynpart_hashjoin_1.q.out    |    12 +-
 .../llap/tez_vector_dynpart_hashjoin_2.q.out    |     6 +-
 .../vectorized_dynamic_partition_pruning.q.out  |   270 +-
 .../clientpositive/llap_partitioned.q.out       |     4 +
 .../clientpositive/llap_uncompressed.q.out      |    12 +
 .../clientpositive/louter_join_ppr.q.out        |     8 +
 ql/src/test/results/clientpositive/merge3.q.out |     5 +
 .../results/clientpositive/metadataonly1.q.out  |    12 +
 .../results/clientpositive/multi_insert.q.out   |    24 +-
 .../multi_insert_lateral_view.q.out             |     2 +
 ...i_insert_move_tasks_share_dependencies.q.out |    24 +-
 .../clientpositive/multi_insert_union_src.q.out |     2 +
 .../clientpositive/multi_join_union.q.out       |     8 +
 .../results/clientpositive/nestedvirtual.q.out  |     9 +
 .../clientpositive/non_ascii_literal2.q.out     |     2 +
 .../results/clientpositive/nullformatCTAS.q.out |     2 +
 .../clientpositive/optimize_nullscan.q.out      |    14 +
 .../results/clientpositive/orc_create.q.out     |    52 +
 .../results/clientpositive/orc_createas1.q.out  |     5 +
 .../results/clientpositive/orc_file_dump.q.out  |     6 +-
 .../test/results/clientpositive/orc_llap.q.out  |    42 +
 .../results/clientpositive/orc_merge10.q.out    |     4 +-
 .../results/clientpositive/orc_merge11.q.out    |     6 +-
 .../clientpositive/orc_merge_incompat3.q.out    |    70 +
 .../results/clientpositive/order_null.q.out     |   222 +
 .../clientpositive/outer_join_ppr.q.java1.7.out |     4 +
 .../clientpositive/parallel_orderby.q.out       |     4 +
 .../results/clientpositive/parquet_ctas.q.out   |     9 +
 .../results/clientpositive/parquet_join.q.out   |     5 +
 .../parquet_map_null.q.java1.7.out              |     1 +
 .../parquet_map_of_arrays_of_ints.q.out         |     1 +
 .../clientpositive/parquet_map_of_maps.q.out    |     1 +
 .../parquet_mixed_partition_formats2.q.out      |     4 +
 .../clientpositive/parquet_nested_complex.q.out |     5 +
 .../parquet_schema_evolution.q.out              |     2 +
 ...arquet_write_correct_definition_levels.q.out |     1 +
 .../clientpositive/partition_decode_name.q.out  |     2 +
 .../clientpositive/partition_special_char.q.out |     2 +
 ql/src/test/results/clientpositive/pcr.q.out    |   220 +-
 ql/src/test/results/clientpositive/pcs.q.out    |    49 +-
 .../results/clientpositive/perf/query12.q.out   |     2 +-
 .../results/clientpositive/perf/query20.q.out   |     2 +-
 .../results/clientpositive/perf/query28.q.out   |   192 +-
 .../results/clientpositive/perf/query51.q.out   |     6 +-
 .../results/clientpositive/perf/query65.q.out   |   116 +-
 .../results/clientpositive/perf/query67.q.out   |     2 +-
 .../results/clientpositive/perf/query70.q.out   |     4 +-
 .../results/clientpositive/perf/query88.q.out   |   912 +-
 .../results/clientpositive/perf/query89.q.out   |     2 +-
 .../results/clientpositive/perf/query98.q.out   |     2 +-
 .../results/clientpositive/pointlookup2.q.out   |   100 +-
 .../results/clientpositive/pointlookup3.q.out   |    96 +-
 .../results/clientpositive/pointlookup4.q.out   |    42 +-
 ql/src/test/results/clientpositive/ppd2.q.out   |    29 +-
 .../results/clientpositive/ppd_clusterby.q.out  |    91 +-
 .../clientpositive/ppd_join_filter.q.out        |    12 +
 .../clientpositive/ppd_outer_join5.q.out        |   127 +-
 .../results/clientpositive/ppd_udf_col.q.out    |    14 +-
 .../results/clientpositive/ppd_union_view.q.out |     4 +
 ql/src/test/results/clientpositive/ppd_vc.q.out |    18 +-
 .../results/clientpositive/ppd_windowing1.q.out |    66 +-
 ql/src/test/results/clientpositive/ptf.q.out    |   144 +-
 .../results/clientpositive/ptf_matchpath.q.out  |    34 +-
 .../results/clientpositive/ptf_streaming.q.out  |    96 +-
 .../results/clientpositive/ptfgroupbyjoin.q.out |     2 +-
 .../test/results/clientpositive/push_or.q.out   |    11 +-
 .../query_result_fileformat.q.out               |     2 +
 .../results/clientpositive/quotedid_basic.q.out |     4 +-
 .../clientpositive/rcfile_createas1.q.out       |     3 +
 .../clientpositive/rcfile_default_format.q.out  |     8 +
 .../reduceSinkDeDuplication_pRS_key_empty.q.out |   220 +
 .../clientpositive/reduce_deduplicate.q.out     |     2 +
 .../test/results/clientpositive/regex_col.q.out |    16 +-
 .../results/clientpositive/regexp_extract.q.out |     2 +
 .../clientpositive/router_join_ppr.q.out        |     8 +
 .../test/results/clientpositive/sample10.q.out  |     7 +-
 .../test/results/clientpositive/sample6.q.out   |    77 +-
 .../test/results/clientpositive/sample8.q.out   |     2 +
 .../sample_islocalmode_hook.q.out               |     4 +
 .../clientpositive/select_same_col.q.out        |     2 +
 .../test/results/clientpositive/semijoin.q.out  |     6 +
 .../test/results/clientpositive/semijoin2.q.out |     4 +-
 .../test/results/clientpositive/semijoin3.q.out |     4 +
 .../test/results/clientpositive/semijoin4.q.out |     2 +-
 .../set_processor_namespaces.q.out              |     2 +-
 .../results/clientpositive/show_functions.q.out |     4 +
 .../clientpositive/skewjoin_noskew.q.out        |     2 +
 .../clientpositive/skewjoin_onesideskew.q.out   |     2 +
 .../results/clientpositive/skiphf_aggr.q.out    |   267 +
 .../results/clientpositive/smb_mapjoin9.q.out   |     4 +
 .../results/clientpositive/smb_mapjoin_11.q.out |     1 +
 .../results/clientpositive/smb_mapjoin_13.q.out |    20 +-
 .../results/clientpositive/smb_mapjoin_15.q.out |    40 +-
 .../clientpositive/sort_merge_join_desc_5.q.out |     1 +
 .../clientpositive/sort_merge_join_desc_6.q.out |     1 +
 .../clientpositive/sort_merge_join_desc_7.q.out |     1 +
 .../spark/auto_join_reordering_values.q.out     |     8 +
 .../clientpositive/spark/auto_join_stats.q.out  |    31 +-
 .../clientpositive/spark/auto_join_stats2.q.out |    31 +-
 .../spark/auto_sortmerge_join_1.q.out           |     3 +
 .../spark/auto_sortmerge_join_12.q.out          |     1 +
 .../spark/auto_sortmerge_join_2.q.out           |     2 +
 .../spark/auto_sortmerge_join_3.q.out           |     3 +
 .../spark/auto_sortmerge_join_4.q.out           |     3 +
 .../spark/auto_sortmerge_join_5.q.out           |     3 +
 .../spark/auto_sortmerge_join_7.q.out           |     3 +
 .../spark/auto_sortmerge_join_8.q.out           |     3 +
 .../results/clientpositive/spark/bucket2.q.out  |     1 +
 .../results/clientpositive/spark/bucket3.q.out  |     1 +
 .../results/clientpositive/spark/bucket4.q.out  |     1 +
 .../results/clientpositive/spark/bucket5.q.out  |     2 +
 .../spark/bucket_map_join_1.q.out               |     1 +
 .../spark/bucket_map_join_2.q.out               |     1 +
 .../spark/bucketizedhiveinputformat.q.out       |     3 +-
 .../clientpositive/spark/bucketmapjoin10.q.out  |     1 +
 .../clientpositive/spark/bucketmapjoin11.q.out  |     2 +
 .../clientpositive/spark/bucketmapjoin12.q.out  |     2 +
 .../clientpositive/spark/bucketmapjoin13.q.out  |     4 +
 .../clientpositive/spark/bucketmapjoin7.q.out   |    19 +-
 .../clientpositive/spark/bucketmapjoin8.q.out   |     2 +
 .../clientpositive/spark/bucketmapjoin9.q.out   |     2 +
 .../spark/column_access_stats.q.out             |    28 +-
 .../spark/cross_product_check_1.q.out           |     4 +
 .../spark/cross_product_check_2.q.out           |     4 +
 .../results/clientpositive/spark/ctas.q.out     |    24 +-
 .../spark/disable_merge_for_bucketing.q.out     |     1 +
 .../spark/filter_join_breaktask.q.out           |     4 +
 .../clientpositive/spark/groupby_map_ppr.q.out  |     1 +
 .../spark/groupby_map_ppr_multi_distinct.q.out  |     1 +
 .../clientpositive/spark/groupby_ppr.q.out      |     1 +
 .../spark/groupby_ppr_multi_distinct.q.out      |     1 +
 .../spark/groupby_resolution.q.out              |     2 +-
 .../spark/groupby_sort_1_23.q.out               |    51 +-
 .../spark/groupby_sort_skew_1_23.q.out          |    58 +-
 .../results/clientpositive/spark/join17.q.out   |     2 +
 .../results/clientpositive/spark/join34.q.out   |     3 +
 .../results/clientpositive/spark/join35.q.out   |     5 +
 .../results/clientpositive/spark/join38.q.out   |     8 +-
 .../results/clientpositive/spark/join41.q.out   |     2 +
 .../results/clientpositive/spark/join9.q.out    |     2 +
 .../spark/join_filters_overlap.q.out            |    19 +
 .../clientpositive/spark/louter_join_ppr.q.out  |     8 +
 .../clientpositive/spark/multi_insert.q.out     |    24 +-
 .../spark/multi_insert_lateral_view.q.out       |     2 +
 ...i_insert_move_tasks_share_dependencies.q.out |    24 +-
 .../clientpositive/spark/multi_join_union.q.out |     8 +
 .../spark/optimize_nullscan.q.out               |    14 +
 .../spark/outer_join_ppr.q.java1.7.out          |     4 +
 .../clientpositive/spark/parallel_orderby.q.out |     4 +
 .../clientpositive/spark/parquet_join.q.out     |     5 +
 .../test/results/clientpositive/spark/pcr.q.out |   220 +-
 .../clientpositive/spark/ppd_join_filter.q.out  |    12 +
 .../clientpositive/spark/ppd_outer_join5.q.out  |    97 +-
 .../test/results/clientpositive/spark/ptf.q.out |   146 +-
 .../clientpositive/spark/ptf_matchpath.q.out    |    34 +-
 .../clientpositive/spark/ptf_streaming.q.out    |    96 +-
 .../spark/reduce_deduplicate.q.out              |     2 +
 .../clientpositive/spark/router_join_ppr.q.out  |     8 +
 .../results/clientpositive/spark/sample10.q.out |     7 +-
 .../results/clientpositive/spark/sample6.q.out  |    77 +-
 .../results/clientpositive/spark/sample8.q.out  |     2 +
 .../results/clientpositive/spark/semijoin.q.out |     6 +
 .../clientpositive/spark/skewjoin_noskew.q.out  |     2 +
 .../clientpositive/spark/smb_mapjoin_11.q.out   |     1 +
 .../clientpositive/spark/smb_mapjoin_12.q.out   |     2 +
 .../clientpositive/spark/smb_mapjoin_13.q.out   |    20 +-
 .../clientpositive/spark/smb_mapjoin_15.q.out   |    40 +-
 .../results/clientpositive/spark/stats5.q.out   |     2 +
 .../clientpositive/spark/subquery_in.q.out      |     4 +-
 .../clientpositive/spark/temp_table_join1.q.out |     2 +
 .../clientpositive/spark/transform_ppr1.q.out   |     1 +
 .../clientpositive/spark/transform_ppr2.q.out   |     1 +
 .../results/clientpositive/spark/union24.q.out  |    14 +
 .../results/clientpositive/spark/union27.q.out  |    52 +-
 .../results/clientpositive/spark/union31.q.out  |     8 +
 .../results/clientpositive/spark/union32.q.out  |     4 +
 .../results/clientpositive/spark/union33.q.out  |     4 +-
 .../clientpositive/spark/union_ppr.q.out        |    38 +-
 .../clientpositive/spark/union_remove_19.q.out  |    20 +-
 .../spark/union_remove_6_subq.q.out             |     2 +-
 .../clientpositive/spark/union_top_level.q.out  |     2 +
 .../spark/vector_between_in.q.out               |     4 +
 .../spark/vector_decimal_aggregate.q.out        |     4 +
 .../spark/vector_outer_join1.q.out              |   100 +-
 .../spark/vector_outer_join2.q.out              |    60 +
 .../spark/vector_outer_join3.q.out              |    60 +
 .../spark/vector_outer_join4.q.out              |   100 +-
 .../spark/vector_outer_join5.q.out              |     8 +
 .../spark/vectorization_decimal_date.q.out      |     4 +
 .../spark/vectorization_short_regress.q.out     |    12 +
 .../clientpositive/spark/vectorized_ptf.q.out   |   747 +-
 .../special_character_in_tabnames_1.q.out       |     2 +
 ql/src/test/results/clientpositive/stats5.q.out |     2 +
 .../clientpositive/str_to_map.q.java1.7.out     |     1 +
 .../results/clientpositive/subquery_in.q.out    |     4 +-
 .../clientpositive/subquery_in_having.q.out     |     2 +-
 .../results/clientpositive/subquery_notin.q.out |    12 +-
 .../subquery_unqualcolumnrefs.q.out             |     8 +-
 .../results/clientpositive/subquery_views.q.out |     8 +-
 .../results/clientpositive/temp_table.q.out     |     2 +
 .../temp_table_display_colstats_tbllvl.q.out    |     1 +
 .../clientpositive/temp_table_join1.q.out       |     2 +
 .../tez/auto_sortmerge_join_1.q.out             |     6 +
 .../tez/auto_sortmerge_join_10.q.out            |    42 +-
 .../tez/auto_sortmerge_join_11.q.out            |     9 +
 .../tez/auto_sortmerge_join_12.q.out            |     4 +
 .../tez/auto_sortmerge_join_2.q.out             |     4 +
 .../tez/auto_sortmerge_join_3.q.out             |     6 +
 .../tez/auto_sortmerge_join_4.q.out             |     6 +
 .../tez/auto_sortmerge_join_5.q.out             |     4 +
 .../tez/auto_sortmerge_join_7.q.out             |     6 +
 .../tez/auto_sortmerge_join_8.q.out             |     6 +
 .../results/clientpositive/tez/bucket2.q.out    |     1 +
 .../results/clientpositive/tez/bucket3.q.out    |     1 +
 .../results/clientpositive/tez/bucket4.q.out    |     1 +
 .../tez/bucket_map_join_tez1.q.out              |   177 +-
 .../tez/cross_product_check_1.q.out             |     4 +
 .../tez/cross_product_check_2.q.out             |   175 +-
 .../test/results/clientpositive/tez/ctas.q.out  |    24 +-
 .../test/results/clientpositive/tez/cte_2.q.out |     1 +
 .../test/results/clientpositive/tez/cte_4.q.out |     1 +
 .../tez/disable_merge_for_bucketing.q.out       |     1 +
 .../tez/dynamic_partition_pruning.q.out         |   174 +-
 .../tez/dynpart_sort_opt_vectorization.q.out    |     8 +-
 .../tez/dynpart_sort_optimization.q.out         |     4 +-
 .../results/clientpositive/tez/empty_join.q.out |     2 +
 .../clientpositive/tez/explainuser_1.q.out      |   132 +-
 .../clientpositive/tez/explainuser_2.q.out      |    36 +-
 .../clientpositive/tez/explainuser_3.q.out      |     2 +
 .../tez/filter_join_breaktask.q.out             |     4 +
 .../tez/hybridgrace_hashjoin_1.q.out            |     4 +
 .../clientpositive/tez/llap_nullscan.q.out      |     4 +
 .../clientpositive/tez/llapdecider.q.out        |     4 +-
 .../clientpositive/tez/mapjoin_mapjoin.q.out    |     2 +
 .../clientpositive/tez/metadataonly1.q.out      |    12 +
 .../clientpositive/tez/optimize_nullscan.q.out  |    14 +
 .../clientpositive/tez/orc_merge10.q.out        |     4 +-
 .../clientpositive/tez/orc_merge11.q.out        |     6 +-
 .../tez/orc_merge_incompat3.q.out               |    70 +
 .../results/clientpositive/tez/order_null.q.out |   222 +
 .../test/results/clientpositive/tez/ptf.q.out   |   144 +-
 .../clientpositive/tez/ptf_matchpath.q.out      |    34 +-
 .../clientpositive/tez/ptf_streaming.q.out      |    96 +-
 .../clientpositive/tez/subquery_in.q.out        |     4 +-
 .../results/clientpositive/tez/temp_table.q.out |     2 +
 .../results/clientpositive/tez/tez_dml.q.out    |     4 +
 .../tez/tez_join_result_complex.q.out           |    34 +
 .../results/clientpositive/tez/tez_union.q.out  |    11 +
 .../clientpositive/tez/transform_ppr1.q.out     |     1 +
 .../clientpositive/tez/transform_ppr2.q.out     |     1 +
 .../clientpositive/tez/unionDistinct_1.q.out    |   103 +-
 .../clientpositive/tez/unionDistinct_2.q.out    |     6 +
 .../clientpositive/tez/union_fast_stats.q.out   |   130 +-
 .../tez/vector_between_columns.q.out            |     4 +
 .../clientpositive/tez/vector_between_in.q.out  |     4 +
 .../tez/vector_char_mapjoin1.q.out              |     6 +
 .../tez/vector_decimal_10_0.q.out               |     1 +
 .../clientpositive/tez/vector_decimal_3.q.out   |     2 +
 .../clientpositive/tez/vector_decimal_6.q.out   |     2 +
 .../tez/vector_decimal_aggregate.q.out          |     4 +
 .../tez/vector_decimal_expressions.q.out        |     3 +
 .../tez/vector_decimal_math_funcs.q.out         |     4 +
 .../tez/vector_groupby_mapjoin.q.out            |    53 +-
 .../tez/vector_grouping_sets.q.out              |    29 +
 .../clientpositive/tez/vector_inner_join.q.out  |   166 +-
 .../tez/vector_interval_mapjoin.q.out           |    26 +
 .../clientpositive/tez/vector_join30.q.out      |     2 +
 .../tez/vector_join_filters.q.out               |     2 +
 .../clientpositive/tez/vector_join_nulls.q.out  |     2 +
 .../tez/vector_leftsemi_mapjoin.q.out           |     6 +
 .../tez/vector_mapjoin_reduce.q.out             |   118 +-
 .../tez/vector_multi_insert.q.out               |     1 +
 .../tez/vector_nullsafe_join.q.out              |     2 +
 .../clientpositive/tez/vector_outer_join1.q.out |   108 +-
 .../clientpositive/tez/vector_outer_join2.q.out |    60 +
 .../clientpositive/tez/vector_outer_join3.q.out |    60 +
 .../clientpositive/tez/vector_outer_join4.q.out |   108 +-
 .../clientpositive/tez/vector_outer_join5.q.out |     8 +
 .../clientpositive/tez/vector_outer_join6.q.out |    12 +
 .../tez/vector_partitioned_date_time.q.out      |     6 +
 .../tez/vector_reduce_groupby_decimal.q.out     |     4 +
 .../tez/vector_varchar_mapjoin1.q.out           |     6 +
 .../tez/vectorization_decimal_date.q.out        |     4 +
 .../tez/vectorization_short_regress.q.out       |    12 +
 .../vectorized_dynamic_partition_pruning.q.out  |   176 +-
 .../clientpositive/tez/vectorized_ptf.q.out     |   745 +-
 .../clientpositive/tez/windowing_gby.q.out      |     2 +-
 .../results/clientpositive/transform_ppr1.q.out |     1 +
 .../results/clientpositive/transform_ppr2.q.out |     1 +
 .../test/results/clientpositive/udf_chr.q.out   |   Bin 0 -> 1476 bytes
 .../results/clientpositive/udf_explode.q.out    |     2 +
 .../clientpositive/udf_format_number.q.out      |   101 +-
 .../results/clientpositive/udf_replace.q.out    |    32 +
 .../clientpositive/udf_unix_timestamp.q.out     |     2 +
 .../results/clientpositive/udtf_explode.q.out   |     4 +
 .../test/results/clientpositive/union22.q.out   |     2 +
 .../test/results/clientpositive/union24.q.out   |    14 +
 .../test/results/clientpositive/union27.q.out   |    52 +-
 .../test/results/clientpositive/union31.q.out   |     8 +
 .../test/results/clientpositive/union32.q.out   |     4 +
 .../test/results/clientpositive/union33.q.out   |     4 +-
 .../clientpositive/unionDistinct_1.q.out        |   100 +-
 .../clientpositive/unionDistinct_2.q.out        |     6 +
 .../clientpositive/union_fast_stats.q.out       |   136 +-
 .../test/results/clientpositive/union_ppr.q.out |    38 +-
 .../clientpositive/union_remove_19.q.out        |    32 +-
 .../clientpositive/union_remove_6_subq.q.out    |     2 +-
 .../clientpositive/union_top_level.q.out        |     2 +
 .../clientpositive/updateAccessTime.q.out       |     2 +
 .../clientpositive/updateBasicStats.q.out       |     2 +
 .../clientpositive/varchar_nested_types.q.out   |     2 +
 .../clientpositive/vector_between_columns.q.out |     4 +
 .../clientpositive/vector_between_in.q.out      |     4 +
 .../clientpositive/vector_char_mapjoin1.q.out   |     6 +
 .../clientpositive/vector_decimal_10_0.q.out    |     1 +
 .../clientpositive/vector_decimal_3.q.out       |     2 +
 .../clientpositive/vector_decimal_6.q.out       |     2 +
 .../vector_decimal_aggregate.q.out              |     4 +
 .../vector_decimal_expressions.q.out            |     3 +
 .../vector_decimal_math_funcs.q.out             |     4 +
 .../clientpositive/vector_grouping_sets.q.out   |    29 +
 .../vector_interval_mapjoin.q.out               |    26 +
 .../results/clientpositive/vector_join30.q.out  |     2 +
 .../clientpositive/vector_join_filters.q.out    |     2 +
 .../clientpositive/vector_join_nulls.q.out      |     2 +
 .../vector_leftsemi_mapjoin.q.out               |     6 +
 .../clientpositive/vector_multi_insert.q.out    |     1 +
 .../clientpositive/vector_nullsafe_join.q.out   |     2 +
 .../clientpositive/vector_outer_join1.q.out     |    60 +
 .../clientpositive/vector_outer_join2.q.out     |    60 +
 .../clientpositive/vector_outer_join3.q.out     |    60 +
 .../clientpositive/vector_outer_join4.q.out     |    60 +
 .../clientpositive/vector_outer_join5.q.out     |     8 +
 .../clientpositive/vector_outer_join6.q.out     |    12 +
 .../vector_partitioned_date_time.q.out          |     6 +
 .../vector_reduce_groupby_decimal.q.out         |     4 +
 .../results/clientpositive/vector_udf1.q.out    |  1640 ++
 .../vector_varchar_mapjoin1.q.out               |     6 +
 .../vectorization_decimal_date.q.out            |     4 +
 .../vectorization_short_regress.q.out           |    12 +
 .../results/clientpositive/vectorized_ptf.q.out |   745 +-
 .../clientpositive/windowing_navfn.q.out        |     2 +
 .../clientpositive/windowing_order_null.q.out   |   183 +
 .../windowing_range_multiorder.q.out            |   910 +
 .../clientpositive/windowing_streaming.q.out    |    12 +-
 serde/if/serde.thrift                           |     1 +
 serde/pom.xml                                   |    22 +
 .../src/gen/thrift/gen-cpp/serde_constants.cpp  |     2 +
 serde/src/gen/thrift/gen-cpp/serde_constants.h  |     1 +
 .../hadoop/hive/serde/serdeConstants.java       |     2 +
 .../org/apache/hadoop/hive/serde/Types.php      |     5 +
 .../org_apache_hadoop_hive_serde/constants.py   |     1 +
 serde/src/gen/thrift/gen-rb/serde_constants.rb  |     2 +
 .../hive/serde2/avro/AvroDeserializer.java      |     2 +-
 .../binarysortable/BinarySortableSerDe.java     |   110 +-
 .../BinarySortableSerDeWithEndPrefix.java       |     2 +-
 .../fast/BinarySortableSerializeWrite.java      |   130 +-
 .../binarysortable/TestBinarySortableFast.java  |    30 +-
 .../binarysortable/TestBinarySortableSerDe.java |    18 +-
 service-rpc/if/TCLIService.thrift               |     9 +
 .../gen/thrift/gen-cpp/TCLIService_types.cpp    |    66 +
 .../src/gen/thrift/gen-cpp/TCLIService_types.h  |    28 +-
 .../rpc/thrift/TGetOperationStatusResp.java     |   312 +-
 service-rpc/src/gen/thrift/gen-php/Types.php    |    69 +
 .../gen-py/TCLIService/TCLIService-remote       |     0
 .../src/gen/thrift/gen-py/TCLIService/ttypes.py |    41 +-
 .../gen/thrift/gen-rb/t_c_l_i_service_types.rb  |     8 +-
 .../org/apache/hive/tmpl/QueryProfileTmpl.jamon |    18 +-
 .../auth/AuthenticationProviderFactory.java     |    12 +-
 .../auth/CustomAuthenticationProviderImpl.java  |    13 +-
 .../hive/service/auth/HiveAuthFactory.java      |    81 +-
 .../auth/LdapAuthenticationProviderImpl.java    |     3 +-
 .../auth/PamAuthenticationProviderImpl.java     |     3 +-
 .../org/apache/hive/service/cli/CLIService.java |     4 +-
 .../hive/service/cli/OperationStatus.java       |    20 +-
 .../hive/service/cli/operation/Operation.java   |    41 +-
 .../service/cli/operation/SQLOperation.java     |    49 +-
 .../service/cli/session/HiveSessionImpl.java    |    21 +-
 .../cli/session/HiveSessionImplwithUGI.java     |     2 +-
 .../thrift/RetryingThriftCLIServiceClient.java  |    26 +-
 .../service/cli/thrift/ThriftCLIService.java    |     3 +
 .../cli/thrift/ThriftCLIServiceClient.java      |     3 +-
 .../cli/thrift/ThriftHttpCLIService.java        |     2 +-
 .../service/cli/thrift/ThriftHttpServlet.java   |    26 +-
 .../apache/hive/service/server/HiveServer2.java |     2 +-
 .../auth/TestLdapAtnProviderWithMiniDS.java     |     2 +-
 .../TestLdapAuthenticationProviderImpl.java     |     2 +-
 .../apache/hive/service/cli/CLIServiceTest.java |   104 +-
 .../cli/TestRetryingThriftCLIServiceClient.java |    15 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |     2 +-
 .../apache/hadoop/hive/thrift/DBTokenStore.java |    49 +-
 .../thrift/DelegationTokenSecretManager.java    |    25 +
 .../hive/thrift/HadoopThriftAuthBridge.java     |   150 +-
 .../hive/thrift/HiveDelegationTokenManager.java |   172 +
 .../TokenStoreDelegationTokenSecretManager.java |    10 -
 .../hadoop/hive/thrift/ZooKeeperTokenStore.java |    16 +-
 spark-client/pom.xml                            |    22 +
 .../apache/hadoop/hive/common/io/DataCache.java |     4 +-
 .../common/io/encoded/EncodedColumnBatch.java   |     9 +-
 .../ql/exec/vector/DecimalColumnVector.java     |    11 +-
 .../hadoop/hive/ql/io/sarg/PredicateLeaf.java   |     1 -
 .../hive/ql/io/sarg/SearchArgumentImpl.java     |     1 -
 875 files changed, 49806 insertions(+), 18346 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2945c3b2/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2945c3b2/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2945c3b2/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2945c3b2/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2945c3b2/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2945c3b2/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2945c3b2/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
----------------------------------------------------------------------


[25/39] hive git commit: Merge remote-tracking branch 'origin/master' into llap

Posted by jd...@apache.org.
Merge remote-tracking branch 'origin/master' into llap


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ffed821c
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ffed821c
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ffed821c

Branch: refs/heads/master
Commit: ffed821cab87cab7d0c7caa8c475c13817251e99
Parents: 53b43cd 0ac424f
Author: Jason Dere <jd...@hortonworks.com>
Authored: Tue Apr 26 10:47:05 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Tue Apr 26 10:47:05 2016 -0700

----------------------------------------------------------------------
 LICENSE                                         |   23 +
 .../java/org/apache/hive/beeline/Commands.java  |    2 +-
 .../org/apache/hadoop/hive/cli/CliDriver.java   |    5 -
 common/pom.xml                                  |    5 +
 .../metrics/metrics2/CodahaleMetrics.java       |   16 +
 .../metrics/metrics2/MetricsReporting.java      |    3 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   42 +-
 .../apache/hadoop/hive/ql/log/PerfLogger.java   |    1 +
 .../hive/common/util/FixedSizedObjectPool.java  |    5 +
 .../java/org/apache/hive/http/HttpServer.java   |   16 +-
 .../hive/hbase/HiveHBaseTableInputFormat.java   |   33 +-
 .../mapreduce/TestHCatMultiOutputFormat.java    |    7 +-
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |    5 +-
 .../apache/hive/jdbc/TestJdbcWithMiniHS2.java   |  108 +-
 itests/qtest/pom.xml                            |    2 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |    9 +-
 .../org/apache/hive/jdbc/HiveBaseResultSet.java |    2 +-
 .../apache/hive/jdbc/HiveDatabaseMetaData.java  |   49 +-
 .../apache/hive/jdbc/HiveResultSetMetaData.java |    2 +-
 .../org/apache/hive/jdbc/HiveStatement.java     |   16 +-
 .../java/org/apache/hive/jdbc/JdbcColumn.java   |    2 +-
 .../llap/registry/impl/LlapRegistryService.java |    6 +-
 .../hive/llap/metrics/LlapMetricsSystem.java    |   57 +
 .../hadoop/hive/llap/metrics/MetricsUtils.java  |   43 +
 .../hadoop/hive/llap/cache/BuddyAllocator.java  |    3 +
 .../hive/llap/cache/EvictionDispatcher.java     |    5 +
 .../hive/llap/cli/LlapStatusServiceDriver.java  |   14 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |    5 +
 .../hive/llap/daemon/impl/LlapTaskReporter.java |   53 +-
 .../llap/daemon/impl/TaskRunnerCallable.java    |    4 +-
 .../hive/llap/io/api/impl/LlapIoImpl.java       |   44 +-
 .../llap/io/decode/EncodedDataConsumer.java     |   15 +-
 .../llap/io/decode/OrcColumnVectorProducer.java |   12 +-
 .../llap/io/decode/OrcEncodedDataConsumer.java  |   52 +-
 .../llap/io/encoded/OrcEncodedDataReader.java   |   83 +-
 .../llap/io/metadata/OrcFileEstimateErrors.java |  121 +
 .../hive/llap/io/metadata/OrcMetadataCache.java |   58 +-
 .../llap/io/metadata/OrcStripeMetadata.java     |    6 +-
 .../hive/llap/metrics/LlapDaemonCacheInfo.java  |    5 +-
 .../llap/metrics/LlapDaemonCacheMetrics.java    |   26 +-
 .../llap/metrics/LlapDaemonExecutorInfo.java    |    6 +
 .../llap/metrics/LlapDaemonExecutorMetrics.java |   45 +-
 .../hive/llap/metrics/LlapDaemonIOInfo.java     |   53 +
 .../hive/llap/metrics/LlapDaemonIOMetrics.java  |  144 ++
 .../hive/llap/metrics/LlapDaemonQueueInfo.java  |   50 -
 .../llap/metrics/LlapDaemonQueueMetrics.java    |  116 -
 .../hive/llap/metrics/LlapMetricsSystem.java    |   57 -
 .../hadoop/hive/llap/metrics/MetricsUtils.java  |   43 -
 .../TestIncrementalObjectSizeEstimator.java     |   31 +-
 .../hive/llap/cache/TestOrcMetadataCache.java   |    2 +-
 .../daemon/impl/TaskExecutorTestHelpers.java    |    6 +-
 .../TestFirstInFirstOutComparator.java          |    4 +-
 .../llap/tezplugins/LlapTaskCommunicator.java   |   14 +-
 .../tezplugins/TestLlapTaskCommunicator.java    |    7 +-
 .../upgrade/derby/hive-schema-2.1.0.derby.sql   |    2 +-
 .../hadoop/hive/metastore/ObjectStore.java      |   41 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |   18 +-
 .../hive/metastore/txn/TestTxnHandler.java      |   15 +
 orc/src/java/org/apache/orc/DataReader.java     |   24 +-
 .../java/org/apache/orc/DataReaderFactory.java  |    9 -
 .../org/apache/orc/MetadataReaderFactory.java   |   12 -
 orc/src/java/org/apache/orc/OrcUtils.java       |   78 +
 orc/src/java/org/apache/orc/Reader.java         |   29 +-
 orc/src/java/org/apache/orc/RecordReader.java   |    8 +-
 .../java/org/apache/orc/TypeDescription.java    |   62 +-
 .../org/apache/orc/impl/BitFieldReader.java     |    5 +-
 .../apache/orc/impl/DataReaderProperties.java   |   41 +-
 .../orc/impl/DefaultMetadataReaderFactory.java  |   14 -
 orc/src/java/org/apache/orc/impl/InStream.java  |    4 +-
 .../java/org/apache/orc/impl/IntegerReader.java |   26 +-
 .../org/apache/orc/impl/MetadataReader.java     |   33 -
 .../org/apache/orc/impl/MetadataReaderImpl.java |  120 -
 .../orc/impl/MetadataReaderProperties.java      |   96 -
 .../apache/orc/impl/RunLengthByteReader.java    |   36 +-
 .../apache/orc/impl/RunLengthIntegerReader.java |   31 +-
 .../orc/impl/RunLengthIntegerReaderV2.java      |   33 +-
 .../java/org/apache/orc/impl/WriterImpl.java    |   47 +-
 .../orc/impl/TestDataReaderProperties.java      |   12 +-
 .../orc/impl/TestMetadataReaderProperties.java  |   72 -
 pom.xml                                         |    3 +-
 ql/pom.xml                                      |    6 +
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  123 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |    3 +-
 .../org/apache/hadoop/hive/ql/QueryState.java   |  114 +
 .../hadoop/hive/ql/exec/ColumnStatsTask.java    |    5 +-
 .../hive/ql/exec/ColumnStatsUpdateTask.java     |    6 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   20 +-
 .../hive/ql/exec/DefaultFetchFormatter.java     |   77 -
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |   37 +-
 .../hadoop/hive/ql/exec/FetchFormatter.java     |   71 -
 .../apache/hadoop/hive/ql/exec/FetchTask.java   |    8 +-
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |   27 +-
 .../hadoop/hive/ql/exec/FunctionTask.java       |    6 +-
 .../hadoop/hive/ql/exec/GroupByOperator.java    |    1 +
 .../hadoop/hive/ql/exec/ListSinkOperator.java   |   11 +-
 .../hadoop/hive/ql/exec/ReduceSinkOperator.java |   15 +-
 .../hadoop/hive/ql/exec/StatsNoJobTask.java     |    5 +-
 .../org/apache/hadoop/hive/ql/exec/Task.java    |    6 +-
 .../hadoop/hive/ql/exec/mr/ExecDriver.java      |   11 +-
 .../hive/ql/exec/mr/HadoopJobExecHelper.java    |   21 +-
 .../hadoop/hive/ql/exec/mr/MapredLocalTask.java |    7 +-
 .../hadoop/hive/ql/exec/spark/SparkTask.java    |    7 +-
 .../hadoop/hive/ql/exec/tez/TezJobMonitor.java  |   83 +-
 .../ql/exec/vector/VectorizedRowBatchCtx.java   |   13 +-
 .../hadoop/hive/ql/history/HiveHistoryImpl.java |    6 +-
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |   10 +-
 .../hadoop/hive/ql/hooks/HookContext.java       |   20 +-
 .../hive/ql/hooks/PostExecutePrinter.java       |   12 +-
 .../hadoop/hive/ql/hooks/PreExecutePrinter.java |   14 +-
 .../hive/ql/index/TableBasedIndexHandler.java   |    2 -
 .../hadoop/hive/ql/io/merge/MergeFileTask.java  |    7 +-
 .../ql/io/orc/DefaultDataReaderFactory.java     |   14 -
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   43 +-
 .../hive/ql/io/orc/OrcRawRecordMerger.java      |    3 +-
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |   65 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |  231 +-
 .../hive/ql/io/orc/RecordReaderUtils.java       |  127 +-
 .../hadoop/hive/ql/io/orc/SchemaEvolution.java  |  234 +-
 .../hive/ql/io/orc/TreeReaderFactory.java       |  838 ++++---
 .../ql/io/orc/VectorizedOrcInputFormat.java     |   32 +-
 .../hadoop/hive/ql/io/orc/WriterImpl.java       |    2 -
 .../ql/io/orc/encoded/EncodedReaderImpl.java    |  106 +-
 .../ql/io/rcfile/stats/PartialScanTask.java     |   12 +-
 .../io/rcfile/truncate/ColumnTruncateTask.java  |    7 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   24 +-
 .../hive/ql/optimizer/ColumnPrunerProcCtx.java  |    2 +-
 .../hive/ql/optimizer/GenMRTableScan1.java      |    2 +-
 .../index/RewriteParseContextGenerator.java     |    7 +-
 .../RewriteQueryUsingAggregateIndexCtx.java     |    2 +-
 .../hive/ql/optimizer/physical/LlapDecider.java |   18 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |  223 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   15 +-
 .../ql/parse/ColumnStatsSemanticAnalyzer.java   |    7 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   13 +-
 .../parse/ExplainSQRewriteSemanticAnalyzer.java |    8 +-
 .../hive/ql/parse/ExplainSemanticAnalyzer.java  |   10 +-
 .../hive/ql/parse/ExportSemanticAnalyzer.java   |    5 +-
 .../hive/ql/parse/FunctionSemanticAnalyzer.java |    5 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |    9 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |   92 +-
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |   11 +-
 .../hive/ql/parse/ImportSemanticAnalyzer.java   |    7 +-
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |    5 +-
 .../hive/ql/parse/MacroSemanticAnalyzer.java    |   12 +-
 .../hadoop/hive/ql/parse/ParseContext.java      |   14 +-
 .../hive/ql/parse/ProcessAnalyzeTable.java      |    2 +-
 .../org/apache/hadoop/hive/ql/parse/QB.java     |    5 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   71 +-
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |   57 +-
 .../hadoop/hive/ql/parse/TaskCompiler.java      |   71 +-
 .../hadoop/hive/ql/parse/TezCompiler.java       |   24 +-
 .../ql/parse/UpdateDeleteSemanticAnalyzer.java  |    5 +-
 .../parse/spark/SparkProcessAnalyzeTable.java   |    2 +-
 .../hadoop/hive/ql/plan/CreateTableDesc.java    |   43 +-
 .../apache/hadoop/hive/ql/plan/ExplainWork.java |   16 -
 .../apache/hadoop/hive/ql/plan/FetchWork.java   |   10 +
 .../hadoop/hive/ql/plan/FileSinkDesc.java       |    9 +
 .../apache/hadoop/hive/ql/plan/PlanUtils.java   |   10 +-
 .../hadoop/hive/ql/session/SessionState.java    |  106 +-
 .../hadoop/hive/ql/exec/TestExecDriver.java     |    7 +-
 .../hive/ql/io/orc/TestRecordReaderImpl.java    |   55 +-
 .../hive/ql/io/orc/TestTypeDescription.java     |    4 +-
 .../hive/ql/io/orc/TestVectorOrcFile.java       | 1647 +++++++-------
 .../hive/ql/io/orc/TestVectorizedORCReader.java |    7 +-
 .../hive/ql/parse/TestHiveDecimalParse.java     |    2 +-
 .../ql/parse/TestMacroSemanticAnalyzer.java     |    7 +-
 .../hadoop/hive/ql/parse/TestQBCompact.java     |    8 +-
 .../ql/parse/TestQBJoinTreeApplyPredicate.java  |    7 +-
 .../hadoop/hive/ql/parse/TestQBSubQuery.java    |    7 +-
 .../ql/parse/TestSemanticAnalyzerFactory.java   |    9 +-
 .../parse/TestUpdateDeleteSemanticAnalyzer.java |   53 +-
 .../authorization/AuthorizationTestUtil.java    |   14 +-
 .../parse/authorization/PrivilegesTestBase.java |    5 +-
 .../TestHiveAuthorizationTaskFactory.java       |   12 +-
 .../parse/authorization/TestPrivilegesV1.java   |    9 +-
 .../parse/authorization/TestPrivilegesV2.java   |    8 +-
 .../create_with_constraints_duplicate_name.q    |    2 +
 .../create_with_constraints_enable.q            |    1 +
 .../create_with_constraints_validate.q          |    1 +
 .../clientpositive/create_with_constraints.q    |   12 +
 ql/src/test/queries/clientpositive/cte_6.q      |   10 +
 .../test/queries/clientpositive/union_offcbo.q  |  483 ++++
 ...create_with_constraints_duplicate_name.q.out |   13 +
 .../create_with_constraints_enable.q.out        |    1 +
 .../create_with_constraints_validate.q.out      |    1 +
 .../alter_partition_coltype.q.out               |  168 --
 .../results/clientpositive/ambiguitycheck.q.out |   30 -
 .../analyze_table_null_partition.q.out          |   16 -
 .../auto_join_reordering_values.q.out           |   83 -
 .../clientpositive/auto_sortmerge_join_1.q.out  |   96 -
 .../clientpositive/auto_sortmerge_join_11.q.out |  153 --
 .../clientpositive/auto_sortmerge_join_12.q.out |   60 -
 .../clientpositive/auto_sortmerge_join_2.q.out  |   64 -
 .../clientpositive/auto_sortmerge_join_3.q.out  |   96 -
 .../clientpositive/auto_sortmerge_join_4.q.out  |   96 -
 .../clientpositive/auto_sortmerge_join_5.q.out  |   96 -
 .../clientpositive/auto_sortmerge_join_7.q.out  |   96 -
 .../clientpositive/auto_sortmerge_join_8.q.out  |   96 -
 .../clientpositive/binary_output_format.q.out   |   36 -
 .../test/results/clientpositive/bucket1.q.out   |   17 -
 .../test/results/clientpositive/bucket2.q.out   |   17 -
 .../test/results/clientpositive/bucket3.q.out   |   21 -
 .../test/results/clientpositive/bucket4.q.out   |   17 -
 .../test/results/clientpositive/bucket5.q.out   |   36 -
 .../results/clientpositive/bucket_many.q.out    |   17 -
 .../clientpositive/bucket_map_join_1.q.out      |   47 -
 .../clientpositive/bucket_map_join_2.q.out      |   47 -
 .../clientpositive/bucket_map_join_spark1.q.out |  104 -
 .../clientpositive/bucket_map_join_spark2.q.out |  104 -
 .../clientpositive/bucket_map_join_spark3.q.out |  104 -
 .../clientpositive/bucket_map_join_spark4.q.out |  134 --
 .../clientpositive/bucketcontext_1.q.out        |   74 -
 .../clientpositive/bucketcontext_2.q.out        |   74 -
 .../clientpositive/bucketcontext_3.q.out        |   74 -
 .../clientpositive/bucketcontext_4.q.out        |   74 -
 .../clientpositive/bucketcontext_5.q.out        |   74 -
 .../clientpositive/bucketcontext_6.q.out        |   74 -
 .../clientpositive/bucketcontext_7.q.out        |   74 -
 .../clientpositive/bucketcontext_8.q.out        |   74 -
 .../results/clientpositive/bucketmapjoin1.q.out |  226 --
 .../clientpositive/bucketmapjoin10.q.out        |   51 -
 .../clientpositive/bucketmapjoin11.q.out        |  112 -
 .../clientpositive/bucketmapjoin12.q.out        |  102 -
 .../clientpositive/bucketmapjoin13.q.out        |  155 --
 .../results/clientpositive/bucketmapjoin2.q.out |  164 --
 .../results/clientpositive/bucketmapjoin3.q.out |  128 --
 .../results/clientpositive/bucketmapjoin4.q.out |  100 -
 .../results/clientpositive/bucketmapjoin5.q.out |  100 -
 .../results/clientpositive/bucketmapjoin7.q.out |   73 -
 .../results/clientpositive/bucketmapjoin8.q.out |  102 -
 .../results/clientpositive/bucketmapjoin9.q.out |  102 -
 .../clientpositive/bucketmapjoin_negative.q.out |   57 -
 .../bucketmapjoin_negative2.q.out               |   50 -
 .../bucketmapjoin_negative3.q.out               |  409 ----
 .../cbo_rp_outer_join_ppr.q.java1.7.out         |  168 --
 .../clientpositive/columnstats_partlvl.q.out    |   30 -
 .../clientpositive/columnstats_tbllvl.q.out     |   27 -
 .../test/results/clientpositive/combine2.q.out  |   23 -
 .../constantPropagateForSubQuery.q.out          |   62 -
 .../create_with_constraints.q.out               |   68 +
 ql/src/test/results/clientpositive/ctas.q.out   |   43 -
 ql/src/test/results/clientpositive/cte_6.q.out  |   86 +
 .../disable_merge_for_bucketing.q.out           |   17 -
 .../display_colstats_tbllvl.q.out               |   13 -
 .../clientpositive/drop_multi_partitions.q.out  |   14 -
 .../dynamic_partition_skip_default.q.out        |   91 -
 .../encryption_join_unencrypted_tbl.q.out       |   32 -
 ...on_join_with_different_encryption_keys.q.out |   32 -
 .../clientpositive/explain_logical.q.out        |  316 ---
 .../extrapolate_part_stats_full.q.out           |   74 -
 .../extrapolate_part_stats_partial.q.out        |  111 -
 .../extrapolate_part_stats_partial_ndv.q.out    |   78 -
 .../clientpositive/filter_join_breaktask.q.out  |  102 -
 .../clientpositive/fouter_join_ppr.q.out        |  336 ---
 .../clientpositive/groupby_map_ppr.q.out        |   71 -
 .../groupby_map_ppr_multi_distinct.q.out        |   88 -
 .../results/clientpositive/groupby_ppr.q.out    |   71 -
 .../groupby_ppr_multi_distinct.q.out            |   88 -
 .../clientpositive/groupby_sort_1_23.q.out      |  782 -------
 .../results/clientpositive/groupby_sort_6.q.out |   90 -
 .../clientpositive/groupby_sort_skew_1_23.q.out |  782 -------
 .../test/results/clientpositive/input23.q.out   |   52 -
 .../test/results/clientpositive/input42.q.out   |   83 -
 .../results/clientpositive/input_part1.q.out    |   56 -
 .../results/clientpositive/input_part2.q.out    |  103 -
 .../results/clientpositive/input_part7.q.out    |   99 -
 .../results/clientpositive/input_part9.q.out    |   29 -
 ql/src/test/results/clientpositive/join17.q.out |   38 -
 ql/src/test/results/clientpositive/join26.q.out |   79 -
 ql/src/test/results/clientpositive/join32.q.out |   73 -
 .../clientpositive/join32_lessSize.q.out        |  330 ---
 ql/src/test/results/clientpositive/join33.q.out |   73 -
 ql/src/test/results/clientpositive/join34.q.out |  104 -
 ql/src/test/results/clientpositive/join35.q.out |  112 -
 ql/src/test/results/clientpositive/join9.q.out  |   54 -
 .../clientpositive/join_filters_overlap.q.out   |  430 ----
 .../results/clientpositive/join_map_ppr.q.out   |  158 --
 .../clientpositive/list_bucket_dml_1.q.out      |   70 -
 .../list_bucket_dml_10.q.java1.7.out            |   28 -
 .../list_bucket_dml_11.q.java1.7.out            |   63 -
 .../list_bucket_dml_12.q.java1.7.out            |  106 -
 .../list_bucket_dml_13.q.java1.7.out            |   70 -
 .../clientpositive/list_bucket_dml_14.q.out     |   42 -
 .../list_bucket_dml_2.q.java1.7.out             |   69 -
 .../clientpositive/list_bucket_dml_3.q.out      |   68 -
 .../list_bucket_dml_4.q.java1.7.out             |  102 -
 .../list_bucket_dml_5.q.java1.7.out             |   76 -
 .../list_bucket_dml_6.q.java1.7.out             |  112 -
 .../clientpositive/list_bucket_dml_7.q.out      |  112 -
 .../list_bucket_dml_8.q.java1.7.out             |   69 -
 .../list_bucket_dml_9.q.java1.7.out             |  102 -
 .../list_bucket_query_multiskew_1.q.out         |  144 --
 .../list_bucket_query_multiskew_2.q.out         |  117 -
 .../list_bucket_query_multiskew_3.q.out         |  130 --
 .../list_bucket_query_oneskew_1.q.out           |   81 -
 .../list_bucket_query_oneskew_2.q.out           |  170 --
 .../list_bucket_query_oneskew_3.q.out           |   28 -
 .../clientpositive/llap/llap_nullscan.q.out     |   20 -
 .../llap/tez_join_result_complex.q.out          |  530 -----
 .../results/clientpositive/load_dyn_part8.q.out |   63 -
 .../clientpositive/louter_join_ppr.q.out        |  336 ---
 ql/src/test/results/clientpositive/macro.q.out  |   62 -
 .../clientpositive/mapjoin_mapjoin.q.out        |   45 -
 ql/src/test/results/clientpositive/merge3.q.out |  110 -
 .../results/clientpositive/metadataonly1.q.out  |  216 --
 .../offset_limit_global_optimizer.q.out         |  288 ---
 .../clientpositive/optimize_nullscan.q.out      |  313 ---
 .../clientpositive/outer_join_ppr.q.java1.7.out |  168 --
 ql/src/test/results/clientpositive/pcr.q.out    |  920 --------
 ql/src/test/results/clientpositive/pcs.q.out    |  517 -----
 .../results/clientpositive/pointlookup2.q.out   |  298 ---
 .../results/clientpositive/pointlookup3.q.out   |  309 ---
 .../results/clientpositive/pointlookup4.q.out   |  146 --
 .../clientpositive/ppd_join_filter.q.out        |  380 ----
 .../results/clientpositive/ppd_union_view.q.out |   42 -
 ql/src/test/results/clientpositive/ppd_vc.q.out |   92 -
 .../clientpositive/ppr_allchildsarenull.q.out   |   88 -
 .../test/results/clientpositive/push_or.q.out   |   42 -
 .../clientpositive/rand_partitionpruner1.q.out  |   22 -
 .../clientpositive/rand_partitionpruner2.q.out  |   33 -
 .../clientpositive/rand_partitionpruner3.q.out  |   94 -
 .../clientpositive/reduce_deduplicate.q.out     |  147 --
 .../clientpositive/router_join_ppr.q.out        |  336 ---
 .../test/results/clientpositive/sample1.q.out   |   39 -
 .../test/results/clientpositive/sample10.q.out  |   39 -
 .../test/results/clientpositive/sample2.q.out   |   23 -
 .../test/results/clientpositive/sample4.q.out   |   25 -
 .../test/results/clientpositive/sample5.q.out   |   25 -
 .../test/results/clientpositive/sample6.q.out   |  256 ---
 .../test/results/clientpositive/sample7.q.out   |   32 -
 .../test/results/clientpositive/sample8.q.out   |   68 -
 .../test/results/clientpositive/sample9.q.out   |   37 -
 .../clientpositive/serde_user_properties.q.out  |  160 --
 .../results/clientpositive/smb_mapjoin9.q.out   |  154 --
 .../results/clientpositive/smb_mapjoin_11.q.out |  101 -
 .../results/clientpositive/smb_mapjoin_12.q.out |  132 --
 .../results/clientpositive/smb_mapjoin_13.q.out |   90 -
 .../results/clientpositive/smb_mapjoin_15.q.out |  210 --
 .../clientpositive/sort_merge_join_desc_5.q.out |   51 -
 .../clientpositive/sort_merge_join_desc_6.q.out |   51 -
 .../clientpositive/sort_merge_join_desc_7.q.out |   51 -
 .../spark/auto_join_reordering_values.q.out     |   83 -
 .../spark/auto_sortmerge_join_1.q.out           |   96 -
 .../spark/auto_sortmerge_join_12.q.out          |   60 -
 .../spark/auto_sortmerge_join_3.q.out           |   96 -
 .../spark/auto_sortmerge_join_4.q.out           |   96 -
 .../spark/auto_sortmerge_join_5.q.out           |   96 -
 .../spark/auto_sortmerge_join_7.q.out           |   96 -
 .../spark/auto_sortmerge_join_8.q.out           |   96 -
 .../results/clientpositive/spark/bucket2.q.out  |   17 -
 .../results/clientpositive/spark/bucket3.q.out  |   21 -
 .../results/clientpositive/spark/bucket4.q.out  |   17 -
 .../results/clientpositive/spark/bucket5.q.out  |   36 -
 .../spark/bucket_map_join_1.q.out               |   47 -
 .../spark/bucket_map_join_2.q.out               |   47 -
 .../spark/bucket_map_join_spark1.q.out          |  104 -
 .../spark/bucket_map_join_spark2.q.out          |  104 -
 .../spark/bucket_map_join_spark3.q.out          |  104 -
 .../spark/bucket_map_join_spark4.q.out          |  134 --
 .../clientpositive/spark/bucketmapjoin1.q.out   |  226 --
 .../clientpositive/spark/bucketmapjoin10.q.out  |   51 -
 .../clientpositive/spark/bucketmapjoin11.q.out  |  112 -
 .../clientpositive/spark/bucketmapjoin12.q.out  |  102 -
 .../clientpositive/spark/bucketmapjoin13.q.out  |  155 --
 .../clientpositive/spark/bucketmapjoin2.q.out   |  164 --
 .../clientpositive/spark/bucketmapjoin3.q.out   |  128 --
 .../clientpositive/spark/bucketmapjoin4.q.out   |  100 -
 .../clientpositive/spark/bucketmapjoin5.q.out   |  100 -
 .../clientpositive/spark/bucketmapjoin7.q.out   |   73 -
 .../clientpositive/spark/bucketmapjoin8.q.out   |  102 -
 .../clientpositive/spark/bucketmapjoin9.q.out   |  102 -
 .../spark/bucketmapjoin_negative.q.out          |   57 -
 .../spark/bucketmapjoin_negative2.q.out         |   50 -
 .../spark/bucketmapjoin_negative3.q.out         |  409 ----
 .../results/clientpositive/spark/ctas.q.out     |   43 -
 .../spark/disable_merge_for_bucketing.q.out     |   17 -
 .../spark/filter_join_breaktask.q.out           |  102 -
 .../clientpositive/spark/groupby_map_ppr.q.out  |   71 -
 .../spark/groupby_map_ppr_multi_distinct.q.out  |   88 -
 .../clientpositive/spark/groupby_ppr.q.out      |   71 -
 .../spark/groupby_ppr_multi_distinct.q.out      |   88 -
 .../spark/groupby_sort_1_23.q.out               |  782 -------
 .../spark/groupby_sort_skew_1_23.q.out          |  782 -------
 .../clientpositive/spark/input_part2.q.out      |  103 -
 .../results/clientpositive/spark/join17.q.out   |   38 -
 .../results/clientpositive/spark/join26.q.out   |   79 -
 .../results/clientpositive/spark/join32.q.out   |   73 -
 .../clientpositive/spark/join32_lessSize.q.out  |  330 ---
 .../results/clientpositive/spark/join33.q.out   |   73 -
 .../results/clientpositive/spark/join34.q.out   |  104 -
 .../results/clientpositive/spark/join35.q.out   |  112 -
 .../results/clientpositive/spark/join9.q.out    |   54 -
 .../spark/join_filters_overlap.q.out            |  430 ----
 .../clientpositive/spark/join_map_ppr.q.out     |  158 --
 .../spark/list_bucket_dml_10.q.java1.7.out      |   28 -
 .../spark/list_bucket_dml_2.q.java1.7.out       |   69 -
 .../clientpositive/spark/load_dyn_part8.q.out   |   63 -
 .../clientpositive/spark/louter_join_ppr.q.out  |  336 ---
 .../clientpositive/spark/mapjoin_mapjoin.q.out  |   45 -
 .../spark/optimize_nullscan.q.out               |  313 ---
 .../spark/outer_join_ppr.q.java1.7.out          |  168 --
 .../test/results/clientpositive/spark/pcr.q.out |  920 --------
 .../clientpositive/spark/ppd_join_filter.q.out  |  380 ----
 .../spark/reduce_deduplicate.q.out              |  147 --
 .../clientpositive/spark/router_join_ppr.q.out  |  336 ---
 .../results/clientpositive/spark/sample1.q.out  |   39 -
 .../results/clientpositive/spark/sample10.q.out |   39 -
 .../results/clientpositive/spark/sample2.q.out  |   23 -
 .../results/clientpositive/spark/sample4.q.out  |   25 -
 .../results/clientpositive/spark/sample5.q.out  |   25 -
 .../results/clientpositive/spark/sample6.q.out  |  256 ---
 .../results/clientpositive/spark/sample7.q.out  |   32 -
 .../results/clientpositive/spark/sample8.q.out  |   68 -
 .../results/clientpositive/spark/sample9.q.out  |   37 -
 .../clientpositive/spark/smb_mapjoin_11.q.out   |  101 -
 .../clientpositive/spark/smb_mapjoin_12.q.out   |  132 --
 .../clientpositive/spark/smb_mapjoin_13.q.out   |   90 -
 .../clientpositive/spark/smb_mapjoin_15.q.out   |  210 --
 .../results/clientpositive/spark/stats0.q.out   |   34 -
 .../results/clientpositive/spark/stats12.q.out  |   14 -
 .../results/clientpositive/spark/stats13.q.out  |   15 -
 .../results/clientpositive/spark/stats3.q.out   |   11 -
 .../clientpositive/spark/transform_ppr1.q.out   |   75 -
 .../clientpositive/spark/transform_ppr2.q.out   |   75 -
 .../results/clientpositive/spark/union22.q.out  |  140 --
 .../results/clientpositive/spark/union24.q.out  |  341 ---
 .../clientpositive/spark/union_ppr.q.out        |   92 -
 .../clientpositive/spark/vectorized_ptf.q.out   | 2128 ------------------
 ql/src/test/results/clientpositive/stats0.q.out |   34 -
 .../test/results/clientpositive/stats11.q.out   |  114 -
 .../test/results/clientpositive/stats12.q.out   |   14 -
 .../test/results/clientpositive/stats13.q.out   |   15 -
 ql/src/test/results/clientpositive/stats3.q.out |   11 -
 .../temp_table_display_colstats_tbllvl.q.out    |   13 -
 .../tez/auto_sortmerge_join_1.q.out             |   96 -
 .../tez/auto_sortmerge_join_11.q.out            |  153 --
 .../tez/auto_sortmerge_join_12.q.out            |   60 -
 .../tez/auto_sortmerge_join_2.q.out             |   64 -
 .../tez/auto_sortmerge_join_3.q.out             |   96 -
 .../tez/auto_sortmerge_join_4.q.out             |   96 -
 .../tez/auto_sortmerge_join_5.q.out             |   96 -
 .../tez/auto_sortmerge_join_7.q.out             |   96 -
 .../tez/auto_sortmerge_join_8.q.out             |   96 -
 .../results/clientpositive/tez/bucket2.q.out    |   17 -
 .../results/clientpositive/tez/bucket3.q.out    |   21 -
 .../results/clientpositive/tez/bucket4.q.out    |   17 -
 .../clientpositive/tez/bucketpruning1.q.out     |  803 -------
 .../test/results/clientpositive/tez/ctas.q.out  |   43 -
 .../tez/disable_merge_for_bucketing.q.out       |   17 -
 .../tez/filter_join_breaktask.q.out             |  102 -
 .../clientpositive/tez/llap_nullscan.q.out      |   20 -
 .../clientpositive/tez/mapjoin_mapjoin.q.out    |   45 -
 .../clientpositive/tez/metadataonly1.q.out      |  216 --
 .../clientpositive/tez/optimize_nullscan.q.out  |  313 ---
 .../results/clientpositive/tez/sample1.q.out    |   39 -
 .../tez/tez_join_result_complex.q.out           |  530 -----
 .../clientpositive/tez/transform_ppr1.q.out     |   75 -
 .../clientpositive/tez/transform_ppr2.q.out     |   75 -
 .../clientpositive/tez/unionDistinct_1.q.out    |  569 -----
 .../clientpositive/tez/vectorized_ptf.q.out     | 2128 ------------------
 .../results/clientpositive/transform_ppr1.q.out |   75 -
 .../results/clientpositive/transform_ppr2.q.out |   75 -
 .../truncate_column_list_bucket.q.out           |   52 -
 .../results/clientpositive/udf_explode.q.out    |  167 --
 .../clientpositive/udf_java_method.q.out        |   63 -
 .../results/clientpositive/udf_reflect.q.out    |   71 -
 .../results/clientpositive/udf_reflect2.q.out   |  229 --
 .../results/clientpositive/udtf_explode.q.out   |  134 --
 .../test/results/clientpositive/union22.q.out   |  140 --
 .../test/results/clientpositive/union24.q.out   |  341 ---
 .../clientpositive/unionDistinct_1.q.out        |  569 -----
 .../results/clientpositive/union_offcbo.q.out   | 1779 +++++++++++++++
 .../test/results/clientpositive/union_ppr.q.out |   92 -
 .../results/clientpositive/vectorized_ptf.q.out | 2128 ------------------
 serde/pom.xml                                   |    5 +
 .../hive/serde2/DefaultFetchFormatter.java      |   73 +
 .../hadoop/hive/serde2/FetchFormatter.java      |   37 +
 .../hadoop/hive/serde2/NoOpFetchFormatter.java  |   48 +
 .../apache/hadoop/hive/serde2/SerDeUtils.java   |    3 +-
 .../hadoop/hive/serde2/thrift/ColumnBuffer.java |  439 ++++
 .../hive/serde2/thrift/ThriftFormatter.java     |   40 +
 .../serde2/thrift/ThriftJDBCBinarySerDe.java    |  178 ++
 .../apache/hadoop/hive/serde2/thrift/Type.java  |  438 ++++
 .../hadoop/hive/serde2/typeinfo/TypeInfo.java   |   14 +-
 service-rpc/if/TCLIService.thrift               |   52 +
 .../src/gen/thrift/gen-cpp/TCLIService.cpp      |  812 ++++++-
 .../src/gen/thrift/gen-cpp/TCLIService.h        |  252 +++
 .../gen-cpp/TCLIService_server.skeleton.cpp     |   10 +
 .../gen/thrift/gen-cpp/TCLIService_types.cpp    | 1016 +++++++--
 .../src/gen/thrift/gen-cpp/TCLIService_types.h  |  296 ++-
 .../hive/service/rpc/thrift/TCLIService.java    | 1716 ++++++++++++++
 .../rpc/thrift/TGetCrossReferenceReq.java       | 1034 +++++++++
 .../rpc/thrift/TGetCrossReferenceResp.java      |  509 +++++
 .../service/rpc/thrift/TGetPrimaryKeysReq.java  |  716 ++++++
 .../service/rpc/thrift/TGetPrimaryKeysResp.java |  509 +++++
 .../apache/hive/service/rpc/thrift/TRowSet.java |  222 +-
 .../src/gen/thrift/gen-php/TCLIService.php      |  432 ++++
 service-rpc/src/gen/thrift/gen-php/Types.php    |  629 ++++++
 .../gen-py/TCLIService/TCLIService-remote       |   14 +
 .../thrift/gen-py/TCLIService/TCLIService.py    |  378 ++++
 .../src/gen/thrift/gen-py/TCLIService/ttypes.py |  445 +++-
 .../src/gen/thrift/gen-rb/t_c_l_i_service.rb    |  108 +
 .../gen/thrift/gen-rb/t_c_l_i_service_types.rb  |   96 +-
 service/pom.xml                                 |    6 +
 .../java/org/apache/hive/http/LlapServlet.java  |  115 +
 .../org/apache/hive/service/cli/CLIService.java |   30 +
 .../org/apache/hive/service/cli/Column.java     |  434 ----
 .../apache/hive/service/cli/ColumnBasedSet.java |   84 +-
 .../hive/service/cli/ColumnDescriptor.java      |   12 +-
 .../apache/hive/service/cli/ColumnValue.java    |    1 +
 .../service/cli/EmbeddedCLIServiceClient.java   |   15 +
 .../apache/hive/service/cli/ICLIService.java    |    8 +
 .../apache/hive/service/cli/RowSetFactory.java  |   17 +-
 .../apache/hive/service/cli/TableSchema.java    |    4 +-
 .../java/org/apache/hive/service/cli/Type.java  |  348 ---
 .../apache/hive/service/cli/TypeDescriptor.java |    1 +
 .../cli/operation/GetCatalogsOperation.java     |    2 +-
 .../cli/operation/GetColumnsOperation.java      |    4 +-
 .../operation/GetCrossReferenceOperation.java   |  169 ++
 .../cli/operation/GetFunctionsOperation.java    |    8 +-
 .../cli/operation/GetPrimaryKeysOperation.java  |  126 ++
 .../cli/operation/GetSchemasOperation.java      |    5 +-
 .../cli/operation/GetTableTypesOperation.java   |    9 +-
 .../cli/operation/GetTablesOperation.java       |    2 +-
 .../cli/operation/GetTypeInfoOperation.java     |    4 +-
 .../cli/operation/HiveCommandOperation.java     |    4 +-
 .../hive/service/cli/operation/Operation.java   |   19 +-
 .../service/cli/operation/OperationManager.java |   31 +-
 .../service/cli/operation/SQLOperation.java     |   65 +-
 .../cli/operation/SQLOperationDisplay.java      |    2 +-
 .../hive/service/cli/session/HiveSession.java   |   28 +
 .../service/cli/session/HiveSessionImpl.java    |   91 +-
 .../thrift/RetryingThriftCLIServiceClient.java  |   16 +
 .../service/cli/thrift/ThriftCLIService.java    |   41 +-
 .../cli/thrift/ThriftCLIServiceClient.java      |   47 +
 .../apache/hive/service/server/HiveServer2.java |    2 +
 .../hive-webapps/hiveserver2/hiveserver2.jsp    |    1 +
 .../hive-webapps/hiveserver2/llap.html          |   47 +
 .../hive-webapps/static/css/json.human.css      |  110 +
 .../hive-webapps/static/js/json.human.js        |  452 ++++
 .../resources/hive-webapps/static/js/llap.js    |   37 +
 .../auth/TestLdapAtnProviderWithMiniDS.java     |  474 +++-
 .../apache/hive/service/cli/CLIServiceTest.java |    2 +-
 .../org/apache/hive/service/cli/TestColumn.java |   14 +-
 .../org/apache/hadoop/hive/common/Pool.java     |    1 +
 .../hive/ql/exec/vector/BytesColumnVector.java  |   13 +-
 .../ql/exec/vector/DecimalColumnVector.java     |    2 +-
 .../hive/ql/exec/vector/DoubleColumnVector.java |    2 +-
 .../hive/ql/exec/vector/LongColumnVector.java   |    2 +-
 .../ql/exec/vector/MultiValuedColumnVector.java |    2 +-
 .../ql/exec/vector/TimestampColumnVector.java   |    2 +-
 .../hive/ql/exec/vector/UnionColumnVector.java  |    2 -
 552 files changed, 19027 insertions(+), 47143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ffed821c/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/ffed821c/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/ffed821c/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/ffed821c/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/ffed821c/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/ffed821c/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/ffed821c/ql/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/ffed821c/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
----------------------------------------------------------------------


[03/39] hive git commit: HIVE-13140. Wire the client to submit execution fragments. (Gunther Hagleitner, Siddharth Seth and Vikram Dixit K)

Posted by jd...@apache.org.
HIVE-13140. Wire the client to submit execution fragments. (Gunther
Hagleitner, Siddharth Seth and Vikram Dixit K)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2e042cc1
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2e042cc1
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2e042cc1

Branch: refs/heads/master
Commit: 2e042cc159c0e7e044297fc4d6b177a8841eb7fd
Parents: f272ace
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Feb 23 23:56:38 2016 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Feb 23 23:56:38 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hive/jdbc/LlapInputFormat.java   |   7 +-
 .../hadoop/hive/llap/LlapInputFormat.java       | 198 ++++++++++++
 .../ext/LlapTaskUmbilicalExternalClient.java    |  23 +-
 .../apache/hadoop/hive/llap/LlapInputSplit.java |  77 ++---
 .../apache/hadoop/hive/llap/SubmitWorkInfo.java |  42 ++-
 .../ql/udf/generic/GenericUDFGetSplits.java     | 317 +++++++++----------
 .../org/apache/tez/dag/api/TaskSpecBuilder.java |  27 +-
 7 files changed, 467 insertions(+), 224 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2e042cc1/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java b/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
index c38dd82..e662414 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/LlapInputFormat.java
@@ -130,8 +130,9 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
 
   @Override
   public RecordReader<NullWritable, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
-    LlapInputSplit llapSplit = (LlapInputSplit)split;
-    return llapSplit.getInputFormat().getRecordReader(llapSplit.getSplit(), job, reporter);
+    try {
+      return ((InputFormat)Class.forName("org.apache.hadoop.hive.llap.LlapInputFormat").newInstance()).getRecordReader(split, job, reporter);
+    } catch (Exception e) { throw new IOException(e); }
   }
 
   @Override
@@ -160,7 +161,7 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
       ResultSet res = stmt.executeQuery(sql);
       while (res.next()) {
         // deserialize split
-        DataInput in = new DataInputStream(new ByteArrayInputStream(res.getBytes(3)));
+        DataInput in = new DataInputStream(res.getBinaryStream(3));
         InputSplit is = (InputSplitWithLocationInfo)Class.forName(res.getString(2)).newInstance(); // todo setAccessible on ctor
         is.readFields(in);
         ins.add(is);

http://git-wip-us.apache.org/repos/asf/hive/blob/2e042cc1/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
new file mode 100644
index 0000000..cf13c1e
--- /dev/null
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.llap;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import com.google.protobuf.ByteString;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
+import org.apache.hadoop.hive.llap.ext.LlapTaskUmbilicalExternalClient;
+import org.apache.hadoop.hive.llap.tez.Converters;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.Token;
+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.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LlapInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LlapInputFormat.class);
+
+
+  public LlapInputFormat() {
+  }
+
+  /*
+   * This proxy record reader has the duty of establishing a connected socket with LLAP, then fire
+   * off the work in the split to LLAP and finally return the connected socket back in an
+   * LlapRecordReader. The LlapRecordReader class reads the results from the socket.
+   */
+  @Override
+  public RecordReader<NullWritable, V> getRecordReader(InputSplit split, JobConf job,
+                                                       Reporter reporter) throws IOException {
+
+    LlapInputSplit llapSplit = (LlapInputSplit) split;
+    SubmitWorkInfo submitWorkInfo = SubmitWorkInfo.fromBytes(llapSplit.getPlanBytes());
+
+    int llapSubmitPort = HiveConf.getIntVar(job, HiveConf.ConfVars.LLAP_DAEMON_RPC_PORT);
+
+    LOG.info("ZZZ: DBG: Starting LlapTaskUmbilicalExternalClient");
+
+    LlapTaskUmbilicalExternalClient llapClient =
+        new LlapTaskUmbilicalExternalClient(job, submitWorkInfo.getTokenIdentifier(),
+            submitWorkInfo.getToken());
+    llapClient.init(job);
+    llapClient.start();
+
+    LOG.info("ZZZ: DBG: Crated LlapClient");
+    // TODO KKK Shutdown the llap client.
+
+    SubmitWorkRequestProto submitWorkRequestProto =
+        constructSubmitWorkRequestProto(submitWorkInfo, llapSplit.getSplitNum(),
+            llapClient.getAddress(), submitWorkInfo.getToken());
+
+    LOG.info("ZZZ: DBG: Created submitWorkRequest for: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString());
+
+    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);
+
+    // this is just the portion that sets up the io to receive data
+    String host = split.getLocations()[0];
+
+    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,
+        conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT));
+
+    LOG.debug("Socket connected");
+
+    socket.getOutputStream().write(id.getBytes());
+    socket.getOutputStream().write(0);
+    socket.getOutputStream().flush();
+
+    LOG.debug("Registered id: " + id);
+
+    return new LlapRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class);
+  }
+
+  @Override
+  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
+    throw new IOException("These are not the splits you are looking for.");
+  }
+
+  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();
+    // This works, assuming the executor is running within YARN.
+    LOG.info("DBG: 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());
+
+    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);
+
+
+    builder.setUsingTezAm(false);
+    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());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2e042cc1/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java b/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
index ecc032d..4305682 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
@@ -1,6 +1,8 @@
 package org.apache.hadoop.hive.llap.ext;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -55,6 +57,7 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
     this.sessionToken = sessionToken;
     // 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
@@ -62,6 +65,7 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
     int numHandlers = HiveConf.getIntVar(conf,
         HiveConf.ConfVars.LLAP_TMP_EXT_CLIENT_NUM_SERVER_HANDLERS);
     llapTaskUmbilicalServer = new LlapTaskUmbilicalServer(conf, umbilical, numHandlers, tokenIdentifier, sessionToken);
+    communicator.start();
   }
 
   @Override
@@ -72,24 +76,31 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
     }
   }
 
+  public InetSocketAddress getAddress() {
+    return llapTaskUmbilicalServer.getAddress();
+  }
+
 
   /**
    * Submit the work for actual execution. This should always have the usingTezAm flag disabled
    * @param submitWorkRequestProto
    */
-  public void submitWork(final SubmitWorkRequestProto submitWorkRequestProto, String llapHost, int llapPort) {
+  public void submitWork(final SubmitWorkRequestProto submitWorkRequestProto, String llapHost, int llapPort, List<TezEvent> tezEvents) {
     Preconditions.checkArgument(submitWorkRequestProto.getUsingTezAm() == false);
 
-    // Store the actual event first. To be returned on the first heartbeat.
-    Event mrInputEvent = null;
-    // Construct a TezEvent out of this, to send it out on the next heaertbeat
 
+    LOG.warn("ZZZ: DBG: " + " Submitting fragment: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString() + " on host: " + llapHost + ", port=" + llapPort);
+//    LOG.info("ZZZ: DBG: " + " Complete SubmitWorkRequest: " + submitWorkRequestProto);
 //    submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString()
 
+    LOG.info("ZZZ: DBG: Received {} events for {}", tezEvents.size(), submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString());
+    // Register the pending events to be sent for this spec.
+    pendingEvents.putIfAbsent(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), tezEvents);
 
     // 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()) {
@@ -110,6 +121,7 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
 
 
 
+
 //    // 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
@@ -157,6 +169,9 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
       LOG.info("ZZZ: DBG: Received heartbeat from taskAttemptId: " + taskAttemptId.toString());
 
       List<TezEvent> tezEvents = pendingEvents.remove(taskAttemptId.toString());
+      if (tezEvents == null) {
+        tezEvents = Collections.emptyList();
+      }
 
       response.setLastRequestId(request.getRequestId());
       // Irrelevant from eventIds. This can be tracked in the AM itself, instead of polluting the task.

http://git-wip-us.apache.org/repos/asf/hive/blob/2e042cc1/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
index 4249a16..d26a579 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
@@ -23,44 +23,26 @@ import java.io.IOException;
 import org.apache.hadoop.hive.metastore.api.Schema;
 import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
 import org.apache.hadoop.mapred.SplitLocationInfo;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.transport.AutoExpandingBuffer;
-import org.apache.thrift.transport.AutoExpandingBufferWriteTransport;
+import org.apache.thrift.TDeserializer;
+import org.apache.thrift.TSerializer;
 
 public class LlapInputSplit implements InputSplitWithLocationInfo {
 
+  int splitNum;
   byte[] planBytes;
   byte[] fragmentBytes;
   SplitLocationInfo[] locations;
   Schema schema;
 
-
-  // // Static
-  // ContainerIdString
-  // DagName
-  // VertexName
-  // FragmentNumber
-  // AttemptNumber - always 0
-  // FragmentIdentifierString - taskAttemptId
-
-  // ProcessorDescsriptor
-  // InputSpec
-  // OutputSpec
-
-  // Tokens
-
-  // // Dynamic
-  //
-
   public LlapInputSplit() {
   }
 
-  public LlapInputSplit(byte[] planBytes, byte[] fragmentBytes, SplitLocationInfo[] locations, Schema schema) {
+  public LlapInputSplit(int splitNum, byte[] planBytes, byte[] fragmentBytes, SplitLocationInfo[] locations, Schema schema) {
     this.planBytes = planBytes;
     this.fragmentBytes = fragmentBytes;
     this.locations = locations;
     this.schema = schema;
+    this.splitNum = splitNum;
   }
 
   public Schema getSchema() {
@@ -81,8 +63,23 @@ public class LlapInputSplit implements InputSplitWithLocationInfo {
     return locs;
   }
 
+  public int getSplitNum() {
+    return splitNum;
+  }
+
+  public byte[] getPlanBytes() {
+    return planBytes;
+  }
+
+  public byte[] getFragmentBytes() {
+    return fragmentBytes;
+  }
+
+
+
   @Override
   public void write(DataOutput out) throws IOException {
+    out.writeInt(splitNum);
     out.writeInt(planBytes.length);
     out.write(planBytes);
 
@@ -97,20 +94,24 @@ public class LlapInputSplit implements InputSplitWithLocationInfo {
     byte[] binarySchema;
 
     try {
-      AutoExpandingBufferWriteTransport transport = new AutoExpandingBufferWriteTransport(1024, 2d);
-      TProtocol protocol = new TBinaryProtocol(transport);
-      schema.write(protocol);
-      binarySchema = transport.getBuf().array();
+      TSerializer serializer = new TSerializer();
+      byte[] serialzied = serializer.serialize(schema);
+      out.writeInt(serialzied.length);
+      out.write(serialzied);
+//      AutoExpandingBufferWriteTransport transport = new AutoExpandingBufferWriteTransport(1024, 2d);
+//      TProtocol protocol = new TBinaryProtocol(transport);
+//      schema.write(protocol);
+//      binarySchema = transport.getBuf().array();
     } catch (Exception e) {
       throw new IOException(e);
     }
 
-    out.writeInt(binarySchema.length);
-    out.write(binarySchema);
+
   }
 
   @Override
   public void readFields(DataInput in) throws IOException {
+    splitNum = in.readInt();
     int length = in.readInt();
     planBytes = new byte[length];
     in.readFully(planBytes);
@@ -129,14 +130,18 @@ public class LlapInputSplit implements InputSplitWithLocationInfo {
     length = in.readInt();
 
     try {
-      AutoExpandingBufferWriteTransport transport =
-          new AutoExpandingBufferWriteTransport(length, 2d);
-      AutoExpandingBuffer buf = transport.getBuf();
-      in.readFully(buf.array(), 0, length);
-
-      TProtocol protocol = new TBinaryProtocol(transport);
+      byte[] schemaBytes = new byte[length];
+      in.readFully(schemaBytes);
+      TDeserializer tDeserializer = new TDeserializer();
       schema = new Schema();
-      schema.read(protocol);
+      tDeserializer.deserialize(schema, schemaBytes);
+//      AutoExpandingBufferReadTransport transport = new AutoExpandingBufferReadTransport(length, 2d);
+//      AutoExpandingBuffer buf = transport.getBuf();
+//      in.readFully(buf.array(), 0, length);
+//
+//      TProtocol protocol = new TBinaryProtocol(transport);
+//      schema = new Schema();
+//      schema.read(protocol);
     } catch (Exception e) {
       throw new IOException(e);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/2e042cc1/ql/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java b/ql/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
index a9a3738..83149ab 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
@@ -6,18 +6,29 @@ import java.io.IOException;
 
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.common.security.JobTokenIdentifier;
+import org.apache.tez.common.security.JobTokenSecretManager;
 import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class SubmitWorkInfo implements Writable {
 
   private TaskSpec taskSpec;
   private ApplicationId fakeAppId;
+  private long creationTime;
 
-  public SubmitWorkInfo(TaskSpec taskSpec, ApplicationId fakeAppId) {
+  // This is used to communicate over the LlapUmbilicalProtocol. Not related to tokens used to
+  // talk to LLAP daemons itself via the securit work.
+  private Token<JobTokenIdentifier> token;
+
+  public SubmitWorkInfo(TaskSpec taskSpec, ApplicationId fakeAppId, long creationTime) {
     this.taskSpec = taskSpec;
     this.fakeAppId = fakeAppId;
+    this.token = createJobToken();
+    this.creationTime = creationTime;
   }
 
   // Empty constructor for writable etc.
@@ -32,11 +43,25 @@ public class SubmitWorkInfo implements Writable {
     return fakeAppId;
   }
 
+  public String getTokenIdentifier() {
+    return fakeAppId.toString();
+  }
+
+  public Token<JobTokenIdentifier> getToken() {
+    return token;
+  }
+
+  public long getCreationTime() {
+    return creationTime;
+  }
+
   @Override
   public void write(DataOutput out) throws IOException {
     taskSpec.write(out);
     out.writeLong(fakeAppId.getClusterTimestamp());
     out.writeInt(fakeAppId.getId());
+    token.write(out);
+    out.writeLong(creationTime);
   }
 
   @Override
@@ -46,6 +71,9 @@ public class SubmitWorkInfo implements Writable {
     long appIdTs = in.readLong();
     int appIdId = in.readInt();
     fakeAppId = ApplicationId.newInstance(appIdTs, appIdId);
+    token = new Token<>();
+    token.readFields(in);
+    creationTime = in.readLong();
   }
 
   public static byte[] toBytes(SubmitWorkInfo submitWorkInfo) throws IOException {
@@ -54,7 +82,7 @@ public class SubmitWorkInfo implements Writable {
     return dob.getData();
   }
 
-  public SubmitWorkInfo fromBytes(byte[] submitWorkInfoBytes) throws IOException {
+  public static SubmitWorkInfo fromBytes(byte[] submitWorkInfoBytes) throws IOException {
     DataInputBuffer dib = new DataInputBuffer();
     dib.reset(submitWorkInfoBytes, 0, submitWorkInfoBytes.length);
     SubmitWorkInfo submitWorkInfo = new SubmitWorkInfo();
@@ -62,4 +90,14 @@ public class SubmitWorkInfo implements Writable {
     return submitWorkInfo;
   }
 
+
+  private Token<JobTokenIdentifier> createJobToken() {
+    String tokenIdentifier = fakeAppId.toString();
+    JobTokenIdentifier identifier = new JobTokenIdentifier(new Text(
+        tokenIdentifier));
+    Token<JobTokenIdentifier> sessionToken = new Token<JobTokenIdentifier>(identifier,
+        new JobTokenSecretManager());
+    sessionToken.setService(identifier.getJobId());
+    return sessionToken;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/2e042cc1/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
index 9c7e1f2..9fa4aa8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGetSplits.java
@@ -18,131 +18,79 @@
 
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import org.apache.hadoop.hive.llap.LlapInputSplit;
-import org.apache.hadoop.hive.llap.SubmitWorkInfo;
-
+import javax.security.auth.login.LoginException;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URISyntaxException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
 import java.util.Set;
-
-import com.google.common.base.Preconditions;
-
-import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
-
-import javax.security.auth.login.LoginException;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.ArrayList;
 import java.util.UUID;
-import java.io.Serializable;
-import java.io.IOException;
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.DataOutput;
 
 import com.esotericsoftware.kryo.Kryo;
-import java.io.ByteArrayOutputStream;
-import java.io.OutputStream;
-import java.io.InputStream;
-
-import org.apache.hadoop.hive.ql.exec.tez.TezProcessor;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.dag.api.TaskSpecBuilder;
-import org.apache.tez.runtime.api.impl.TaskSpec;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hadoop.hive.ql.udf.UDFType;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.llap.LlapInputSplit;
+import org.apache.hadoop.hive.llap.LlapOutputFormat;
+import org.apache.hadoop.hive.llap.SubmitWorkInfo;
+import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.QueryPlan;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.exec.tez.TezTask;
-import org.apache.hadoop.hive.ql.plan.TezWork;
-import org.apache.hadoop.hive.ql.plan.MapWork;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.MapredContext;
+import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
+import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.exec.MapredContext;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.exec.tez.DagUtils;
+import org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator;
+import org.apache.hadoop.hive.ql.exec.tez.TezTask;
 import org.apache.hadoop.hive.ql.metadata.Hive;
-import org.apache.hadoop.hive.llap.LlapInputFormat;
-import org.apache.hadoop.hive.llap.LlapOutputFormat;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.JobConfigurable;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.MapWork;
+import org.apache.hadoop.hive.ql.plan.TezWork;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.UDFType;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-import org.apache.hadoop.hive.metastore.api.Schema;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URISyntaxException;
-import java.io.FileNotFoundException;
-import java.util.UUID;
-
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.commons.io.FilenameUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapred.SplitLocationInfo;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.SplitLocationInfo;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.tez.dag.api.DAG;
-import org.apache.tez.dag.api.Vertex;
-import org.apache.tez.runtime.api.Event;
-import org.apache.hadoop.hive.ql.Context;
-import org.apache.hadoop.hive.ql.exec.tez.DagUtils;
-import org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator;
-import org.apache.hadoop.hive.ql.plan.MapWork;
-import org.apache.hadoop.hive.ql.plan.TezWork;
-import org.apache.hadoop.hive.metastore.api.Schema;
-import org.apache.hadoop.hive.llap.configuration.LlapConfiguration;
 import org.apache.tez.dag.api.TaskLocationHint;
-import org.apache.tez.dag.api.VertexLocationHint;
-import org.apache.tez.dag.api.event.VertexStateUpdate;
-import org.apache.tez.mapreduce.hadoop.InputSplitInfoMem;
-import org.apache.tez.mapreduce.hadoop.MRInputHelpers;
-import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRInputUserPayloadProto;
-import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitProto;
-import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitsProto;
+import org.apache.tez.dag.api.TaskSpecBuilder;
+import org.apache.tez.dag.api.Vertex;
 import org.apache.tez.runtime.api.Event;
-import org.apache.tez.runtime.api.InputInitializer;
-import org.apache.tez.runtime.api.InputInitializerContext;
-import org.apache.tez.runtime.api.InputSpecUpdate;
 import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
-import org.apache.tez.runtime.api.events.InputDataInformationEvent;
-import org.apache.tez.runtime.api.events.InputInitializerEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * GenericUDFGetSplits.
@@ -155,6 +103,8 @@ public class GenericUDFGetSplits extends GenericUDF {
 
   private static final Logger LOG = LoggerFactory.getLogger(GenericUDFGetSplits.class);
 
+  private static final String LLAP_INTERNAL_INPUT_FORMAT_NAME = "org.apache.hadoop.hive.llap.LlapInputFormat";
+
   private transient StringObjectInspector stringOI;
   private transient IntObjectInspector intOI;
   private final ArrayList<Object> retArray = new ArrayList<Object>();
@@ -190,13 +140,13 @@ public class GenericUDFGetSplits extends GenericUDF {
     } else if (!(arguments[0] instanceof StringObjectInspector)) {
       LOG.error("Got "+arguments[0].getTypeName()+" instead of string.");
       throw new UDFArgumentTypeException(0, "\""
-	  + "string\" is expected at function GET_SPLITS, " + "but \""
-	  + arguments[0].getTypeName() + "\" is found");
+          + "string\" is expected at function GET_SPLITS, " + "but \""
+          + arguments[0].getTypeName() + "\" is found");
     } else if (!(arguments[1] instanceof IntObjectInspector)) {
       LOG.error("Got "+arguments[1].getTypeName()+" instead of int.");
       throw new UDFArgumentTypeException(1, "\""
-	  + "int\" is expected at function GET_SPLITS, " + "but \""
-	  + arguments[1].getTypeName() + "\" is found");
+          + "int\" is expected at function GET_SPLITS, " + "but \""
+          + arguments[1].getTypeName() + "\" is found");
     }
 
     stringOI = (StringObjectInspector) arguments[0];
@@ -204,9 +154,9 @@ public class GenericUDFGetSplits extends GenericUDF {
 
     List<String> names = Arrays.asList("if_class","split_class","split");
     List<ObjectInspector> fieldOIs = Arrays.<ObjectInspector>asList(
-								    PrimitiveObjectInspectorFactory.javaStringObjectInspector,
-								    PrimitiveObjectInspectorFactory.javaStringObjectInspector,
-								    PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector);
+                                                                    PrimitiveObjectInspectorFactory.javaStringObjectInspector,
+                                                                    PrimitiveObjectInspectorFactory.javaStringObjectInspector,
+                                                                    PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector);
     ObjectInspector outputOI = ObjectInspectorFactory.getStandardStructObjectInspector(names, fieldOIs);
     ObjectInspector listOI = ObjectInspectorFactory.getStandardListObjectInspector(outputOI);
     bos = new ByteArrayOutputStream(1024);
@@ -233,80 +183,85 @@ public class GenericUDFGetSplits extends GenericUDF {
       throw new HiveException("Need configuration");
     }
 
-    LOG.info("setting fetch.task.conversion to none and query file format to \""+LlapOutputFormat.class.toString()+"\"");
-    HiveConf.setVar(conf, ConfVars.HIVEFETCHTASKCONVERSION, "none");
-    HiveConf.setVar(conf, HiveConf.ConfVars.HIVEQUERYRESULTFILEFORMAT, LlapOutputFormat.class.toString());
+    String fetchTaskConversion = HiveConf.getVar(conf, ConfVars.HIVEFETCHTASKCONVERSION);
+    String queryResultFormat = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYRESULTFILEFORMAT);
 
-    cpr = driver.compileAndRespond(query);
-    if(cpr.getResponseCode() != 0) {
-      throw new HiveException("Failed to compile query: "+cpr.getException());
-    }
+    try {
+      LOG.info("setting fetch.task.conversion to none and query file format to \""+LlapOutputFormat.class.getName()+"\"");
+      HiveConf.setVar(conf, ConfVars.HIVEFETCHTASKCONVERSION, "none");
+      HiveConf.setVar(conf, HiveConf.ConfVars.HIVEQUERYRESULTFILEFORMAT, LlapOutputFormat.class.getName());
 
-    QueryPlan plan = driver.getPlan();
-    List<Task<?>> roots = plan.getRootTasks();
-    Schema schema = plan.getResultSchema();
+      cpr = driver.compileAndRespond(query);
+      if(cpr.getResponseCode() != 0) {
+        throw new HiveException("Failed to compile query: "+cpr.getException());
+      }
 
-    if (roots == null || roots.size() != 1 || !(roots.get(0) instanceof TezTask)) {
-      throw new HiveException("Was expecting a single TezTask.");
-    }
+      QueryPlan plan = driver.getPlan();
+      List<Task<?>> roots = plan.getRootTasks();
+      Schema schema = plan.getResultSchema();
 
-    Path data = null;
-    String ifc = null;
+      if (roots == null || roots.size() != 1 || !(roots.get(0) instanceof TezTask)) {
+        throw new HiveException("Was expecting a single TezTask.");
+      }
 
-    TezWork tezWork = ((TezTask)roots.get(0)).getWork();
+      Path data = null;
 
-    if (tezWork.getAllWork().size() != 1) {
+      TezWork tezWork = ((TezTask)roots.get(0)).getWork();
 
-      String tableName = "table_"+UUID.randomUUID().toString().replaceAll("[^A-Za-z0-9 ]", "");
+      if (tezWork.getAllWork().size() != 1) {
 
-      String ctas = "create temporary table "+tableName+" as "+query;
-      LOG.info("CTAS: "+ctas);
+        String tableName = "table_"+UUID.randomUUID().toString().replaceAll("[^A-Za-z0-9 ]", "");
 
-      try {
-        cpr = driver.run(ctas, false);
-      } catch(CommandNeedRetryException e) {
-        throw new HiveException(e);
-      }
+        String ctas = "create temporary table "+tableName+" as "+query;
+        LOG.info("CTAS: "+ctas);
 
-      if(cpr.getResponseCode() != 0) {
-        throw new HiveException("Failed to create temp table: " + cpr.getException());
-      }
+        try {
+          cpr = driver.run(ctas, false);
+        } catch(CommandNeedRetryException e) {
+          throw new HiveException(e);
+        }
 
-      query = "select * from " + tableName;
-      cpr = driver.compileAndRespond(query);
-      if(cpr.getResponseCode() != 0) {
-        throw new HiveException("Failed to create temp table: "+cpr.getException());
-      }
+        if(cpr.getResponseCode() != 0) {
+          throw new HiveException("Failed to create temp table: " + cpr.getException());
+        }
 
-      plan = driver.getPlan();
-      roots = plan.getRootTasks();
-      schema = plan.getResultSchema();
+        query = "select * from " + tableName;
+        cpr = driver.compileAndRespond(query);
+        if(cpr.getResponseCode() != 0) {
+          throw new HiveException("Failed to create temp table: "+cpr.getException());
+        }
 
-      if (roots == null || roots.size() != 1 || !(roots.get(0) instanceof TezTask)) {
-        throw new HiveException("Was expecting a single TezTask.");
-      }
+        plan = driver.getPlan();
+        roots = plan.getRootTasks();
+        schema = plan.getResultSchema();
 
-      tezWork = ((TezTask)roots.get(0)).getWork();
-    }
+        if (roots == null || roots.size() != 1 || !(roots.get(0) instanceof TezTask)) {
+          throw new HiveException("Was expecting a single TezTask.");
+        }
 
-    MapWork w = (MapWork)tezWork.getAllWork().get(0);
-    ifc = LlapInputFormat.class.toString();
+        tezWork = ((TezTask)roots.get(0)).getWork();
+      }
 
-    try {
-      for (InputSplit s: getSplits(jc, num, tezWork, schema)) {
-        Object[] os = new Object[3];
-        os[0] = ifc;
-        os[1] = s.getClass().toString();
-        bos.reset();
-        s.write(dos);
-        byte[] frozen = bos.toByteArray();
-        os[2] = frozen;
-        retArray.add(os);
+      MapWork w = (MapWork)tezWork.getAllWork().get(0);
+
+      try {
+        for (InputSplit s: getSplits(jc, num, tezWork, schema)) {
+          Object[] os = new Object[3];
+          os[0] = LLAP_INTERNAL_INPUT_FORMAT_NAME;
+          os[1] = s.getClass().getName();
+          bos.reset();
+          s.write(dos);
+          byte[] frozen = bos.toByteArray();
+          os[2] = frozen;
+          retArray.add(os);
+        }
+      } catch(Exception e) {
+        throw new HiveException(e);
       }
-    } catch(Exception e) {
-      throw new HiveException(e);
+    } finally {
+      HiveConf.setVar(conf, ConfVars.HIVEFETCHTASKCONVERSION, fetchTaskConversion);
+      HiveConf.setVar(conf, HiveConf.ConfVars.HIVEQUERYRESULTFILEFORMAT, queryResultFormat);
     }
-
     return retArray;
   }
 
@@ -332,6 +287,7 @@ public class GenericUDFGetSplits extends GenericUDF {
       dag.addVertex(wx);
       utils.addCredentials(mapWork, dag);
 
+
       // we have the dag now proceed to get the splits:
       HiveSplitGenerator splitGenerator = new HiveSplitGenerator(null);
       Preconditions.checkState(HiveConf.getBoolVar(wxConf,
@@ -342,8 +298,8 @@ public class GenericUDFGetSplits extends GenericUDF {
       List<Event> eventList = splitGenerator.initialize();
 
       // hack - just serializing with kryo for now. This needs to be done properly
-      InputSplit[] result = new InputSplit[eventList.size()];
-      ByteArrayOutputStream bos = new ByteArrayOutputStream(10240);
+      InputSplit[] result = new InputSplit[eventList.size() - 1];
+      DataOutputBuffer dob = new DataOutputBuffer();
 
       InputConfigureVertexTasksEvent configureEvent = (InputConfigureVertexTasksEvent) eventList.get(0);
 
@@ -351,11 +307,25 @@ public class GenericUDFGetSplits extends GenericUDF {
 
       Preconditions.checkState(hints.size() == eventList.size() -1);
 
+      LOG.error("DBG: NumEvents=" + eventList.size());
+      LOG.error("DBG: NumSplits=" + result.length);
+
+      ApplicationId fakeApplicationId = ApplicationId.newInstance(Math.abs(new Random().nextInt()), 0);
+      TaskSpec taskSpec =
+          new TaskSpecBuilder().constructTaskSpec(dag, vertexName, eventList.size() - 1, fakeApplicationId);
+
+      SubmitWorkInfo submitWorkInfo =
+          new SubmitWorkInfo(taskSpec, fakeApplicationId, System.currentTimeMillis());
+      EventMetaData sourceMetaData =
+          new EventMetaData(EventMetaData.EventProducerConsumerType.INPUT, vertexName,
+              "NULL_VERTEX", null);
+      EventMetaData destinationMetaInfo = new TaskSpecBuilder().getDestingationMetaData(wx);
+
       LOG.info("DBG: Number of splits: " + (eventList.size() - 1));
-      for (int i = 1 ; i < eventList.size() ; i++) {
+      for (int i = 0; i < eventList.size() - 1; i++) {
         // Creating the TezEvent here itself, since it's easy to serialize.
-        Event event = eventList.get(i);
-        TaskLocationHint hint = hints.get(i-1);
+        Event event = eventList.get(i + 1);
+        TaskLocationHint hint = hints.get(i);
         Set<String> hosts = hint.getHosts();
         LOG.info("DBG: Using locations: " + hosts.toString());
         if (hosts.size() != 1) {
@@ -367,18 +337,17 @@ public class GenericUDFGetSplits extends GenericUDF {
         for (String host : hosts) {
           locations[j++] = new SplitLocationInfo(host, false);
         }
+        TezEvent tezEvent = new TezEvent(event, sourceMetaData, System.currentTimeMillis());
+        tezEvent.setDestinationInfo(destinationMetaInfo);
 
         bos.reset();
-        Kryo kryo = SerializationUtilities.borrowKryo();
-        SerializationUtilities.serializeObjectByKryo(kryo, event, bos);
-        SerializationUtilities.releaseKryo(kryo);
+        dob.reset();
+        tezEvent.write(dob);
 
-        TaskSpec taskSpec = new TaskSpecBuilder().constructTaskSpec(dag, vertexName, eventList.size() - 1);
-        ApplicationId fakeApplicationId = ApplicationId.newInstance(new Random().nextInt(), 0);
-        SubmitWorkInfo submitWorkInfo = new SubmitWorkInfo(taskSpec, fakeApplicationId);
         byte[] submitWorkBytes = SubmitWorkInfo.toBytes(submitWorkInfo);
 
-        result[i-1] = new LlapInputSplit(submitWorkBytes, bos.toByteArray(), locations, schema);
+        result[i] =
+            new LlapInputSplit(i, submitWorkBytes, dob.getData(), locations, schema);
       }
       return result;
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/2e042cc1/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java b/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
index d0c7c5a..5cabb6a 100644
--- a/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
+++ b/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
@@ -4,6 +4,12 @@ import java.util.ArrayList;
 import java.util.List;
 
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+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.EventMetaData;
 import org.apache.tez.runtime.api.impl.InputSpec;
 import org.apache.tez.runtime.api.impl.OutputSpec;
 import org.apache.tez.runtime.api.impl.TaskSpec;
@@ -11,7 +17,7 @@ import org.apache.tez.runtime.api.impl.TaskSpec;
 // Proxy class within the tez.api package to access package private methods.
 public class TaskSpecBuilder {
 
-  public TaskSpec constructTaskSpec(DAG dag, String vertexName, int numSplits) {
+  public TaskSpec constructTaskSpec(DAG dag, String vertexName, int numSplits, ApplicationId appId) {
     Vertex vertex = dag.getVertex(vertexName);
     ProcessorDescriptor processorDescriptor = vertex.getProcessorDescriptor();
     List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> inputs =
@@ -35,11 +41,22 @@ public class TaskSpecBuilder {
       outputSpecs.add(outputSpec);
     }
 
-    TaskSpec taskSpec = TaskSpec
-        .createBaseTaskSpec(dag.getName(), vertexName, numSplits, processorDescriptor, inputSpecs,
-            outputSpecs, null);
+    TezDAGID dagId = TezDAGID.getInstance(appId, 0);
+    TezVertexID vertexId = TezVertexID.getInstance(dagId, 0);
+    TezTaskID taskId = TezTaskID.getInstance(vertexId, 0);
+    TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 0);
+    return new TaskSpec(taskAttemptId, dag.getName(), vertexName, numSplits, processorDescriptor, inputSpecs, outputSpecs, null);
+  }
 
-    return taskSpec;
+  public EventMetaData getDestingationMetaData(Vertex vertex) {
+    List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> inputs =
+        vertex.getInputs();
+    Preconditions.checkState(inputs.size() == 1);
+    String inputName = inputs.get(0).getName();
+    EventMetaData destMeta =
+        new EventMetaData(EventMetaData.EventProducerConsumerType.INPUT, vertex.getName(),
+            inputName, null);
+    return destMeta;
   }
 
 }


[18/39] hive git commit: HIVE-13529: Move around some of the classes created during llap branch work

Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/llap-common/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java b/llap-common/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
deleted file mode 100644
index 4e000ff..0000000
--- a/llap-common/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
+++ /dev/null
@@ -1,155 +0,0 @@
-package org.apache.hadoop.hive.llap;
-
-import com.google.common.base.Preconditions;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.JobConf;
-
-import org.apache.hadoop.hive.llap.Row;
-import org.apache.hadoop.hive.llap.FieldDesc;
-import org.apache.hadoop.hive.llap.Schema;
-import org.apache.hadoop.hive.llap.TypeDesc;
-import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.serde2.SerDe;
-import org.apache.hadoop.hive.serde2.SerDeException;
-import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
-import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
-import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.StructField;
-import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-public class LlapRowRecordReader implements RecordReader<NullWritable, Row> {
-
-  private static final Logger LOG = LoggerFactory.getLogger(LlapRowRecordReader.class);
-
-  Configuration conf;
-  RecordReader<NullWritable, Text> reader;
-  Schema schema;
-  SerDe serde;
-  final Text textData = new Text();
-
-  public LlapRowRecordReader(Configuration conf, Schema schema, RecordReader<NullWritable, Text> reader) {
-    this.conf = conf;
-    this.schema = schema;
-    this.reader = reader;
-  }
-
-  @Override
-  public void close() throws IOException {
-    reader.close();
-  }
-
-  @Override
-  public NullWritable createKey() {
-    return NullWritable.get();
-  }
-
-  @Override
-  public Row createValue() {
-    return new Row(schema);
-  }
-
-  @Override
-  public long getPos() throws IOException {
-    return 0;
-  }
-
-  @Override
-  public float getProgress() throws IOException {
-    return 0;
-  }
-
-  @Override
-  public boolean next(NullWritable key, Row value) throws IOException {
-    Preconditions.checkArgument(value != null);
-
-    if (serde == null) {
-      try {
-        serde = initSerDe(conf);
-      } catch (SerDeException err) {
-        throw new IOException(err);
-      }
-    }
-
-    boolean hasNext = reader.next(key,  textData);
-    if (hasNext) {
-      // Deserialize Text to column values, and populate the row record
-      Object rowObj;
-      try {
-        StructObjectInspector rowOI = (StructObjectInspector) serde.getObjectInspector();
-        rowObj = serde.deserialize(textData);
-        List<? extends StructField> colFields = rowOI.getAllStructFieldRefs();
-        for (int idx = 0; idx < colFields.size(); ++idx) {
-          StructField field = colFields.get(idx);
-          Object colValue = rowOI.getStructFieldData(rowObj, field);
-          Preconditions.checkState(field.getFieldObjectInspector().getCategory() == Category.PRIMITIVE,
-              "Cannot handle non-primitive column type " + field.getFieldObjectInspector().getTypeName());
-
-          PrimitiveObjectInspector poi = (PrimitiveObjectInspector) field.getFieldObjectInspector();
-          // char/varchar special cased here since the row record handles them using Text
-          switch (poi.getPrimitiveCategory()) {
-            case CHAR:
-              value.setValue(idx, ((HiveCharWritable) poi.getPrimitiveWritableObject(colValue)).getPaddedValue());
-              break;
-            case VARCHAR:
-              value.setValue(idx, ((HiveVarcharWritable) poi.getPrimitiveWritableObject(colValue)).getTextValue());
-              break;
-            default:
-              value.setValue(idx, (Writable) poi.getPrimitiveWritableObject(colValue));
-              break;
-          }
-        }
-      } catch (SerDeException err) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Error deserializing row from text: " + textData);
-        }
-        throw new IOException("Error deserializing row data", err);
-      }
-    }
-
-    return hasNext;
-  }
-
-  public Schema getSchema() {
-    return schema;
-  }
-
-  protected SerDe initSerDe(Configuration conf) throws SerDeException {
-    Properties props = new Properties();
-    StringBuffer columnsBuffer = new StringBuffer();
-    StringBuffer typesBuffer = new StringBuffer();
-    boolean isFirst = true;
-    for (FieldDesc colDesc : schema.getColumns()) {
-      if (!isFirst) {
-        columnsBuffer.append(',');
-        typesBuffer.append(',');
-      }
-      columnsBuffer.append(colDesc.getName());
-      typesBuffer.append(colDesc.getTypeDesc().toString());
-      isFirst = false;
-    }
-    String columns = columnsBuffer.toString();
-    String types = typesBuffer.toString();
-    props.put(serdeConstants.LIST_COLUMNS, columns);
-    props.put(serdeConstants.LIST_COLUMN_TYPES, types);
-    SerDe serde = new LazySimpleSerDe();
-    serde.initialize(conf, props);
-
-    return serde;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/llap-ext-client/pom.xml
----------------------------------------------------------------------
diff --git a/llap-ext-client/pom.xml b/llap-ext-client/pom.xml
new file mode 100644
index 0000000..5a7e385
--- /dev/null
+++ b/llap-ext-client/pom.xml
@@ -0,0 +1,140 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hive</groupId>
+    <artifactId>hive</artifactId>
+    <version>2.1.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>hive-llap-ext-client</artifactId>
+  <packaging>jar</packaging>
+  <name>Hive Llap External Client</name>
+
+  <properties>
+    <hive.path.to.root>..</hive.path.to.root>
+  </properties>
+
+  <dependencies>
+    <!-- dependencies are always listed in sorted order by groupId, artifactId -->
+    <!-- intra-project -->
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-jdbc</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-llap-client</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <!-- inter-project -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <version>${hadoop.version}</version>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commmons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>${hadoop.version}</version>
+      <optional>true</optional>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-registry</artifactId>
+      <version>${hadoop.version}</version>
+      <optional>true</optional>
+    </dependency>
+    <!-- test inter-project -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <version>${commons-lang3.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>${mockito-all.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <version>${hadoop.version}</version>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commmons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+  </dependencies>
+  <build>
+    <sourceDirectory>${basedir}/src/java</sourceDirectory>
+    <testSourceDirectory>${basedir}/src/test</testSourceDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>src/gen/protobuf/gen-java</source>
+                <source>src/gen/thrift/gen-javabean</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/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
new file mode 100644
index 0000000..61eb2ea
--- /dev/null
+++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.llap;
+
+import java.util.ArrayList;
+import java.util.List;
+
+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 org.apache.hadoop.fs.FileSystem;
+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.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hive.llap.ext.LlapInputSplit;
+
+import com.google.common.base.Preconditions;
+
+public class LlapBaseInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
+
+  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;
+    return llapSplit.getInputFormat().getRecordReader(llapSplit.getSplit(), job, reporter);
+  }
+
+  @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(3));
+        InputSplitWithLocationInfo is = (InputSplitWithLocationInfo)Class.forName(res.getString(2)).newInstance();
+        is.readFields(in);
+        ins.add(new LlapInputSplit(is, res.getString(1)));
+      }
+
+      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
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
----------------------------------------------------------------------
diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
new file mode 100644
index 0000000..ce419af
--- /dev/null
+++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
@@ -0,0 +1,165 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.llap;
+
+import java.io.OutputStream;
+import java.io.InputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.FileInputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.io.RCFile.Reader;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.RecordWriter;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.hive.llap.io.api.LlapProxy;
+import org.apache.hadoop.hive.llap.LlapBaseRecordReader;
+import org.apache.hadoop.hive.llap.Schema;
+
+import org.apache.hadoop.hive.llap.LlapBaseInputFormat;
+
+public class LlapDump {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LlapDump.class);
+
+  private static String url = "jdbc:hive2://localhost:10000/default";
+  private static String user = "hive";
+  private static String pwd = "";
+  private static String query = "select * from test";
+  private static String numSplits = "1";
+
+  public static void main(String[] args) throws Exception {
+    Options opts = createOptions();
+    CommandLine cli = new GnuParser().parse(opts, args);
+
+    if (cli.hasOption('h')) {
+      HelpFormatter formatter = new HelpFormatter();
+      formatter.printHelp("orcfiledump", opts);
+      return;
+    }
+
+    if (cli.hasOption('l')) {
+      url = cli.getOptionValue("l");
+    }
+
+    if (cli.hasOption('u')) {
+      user = cli.getOptionValue("u");
+    }
+
+    if (cli.hasOption('p')) {
+      pwd = cli.getOptionValue("p");
+    }
+
+    if (cli.hasOption('n')) {
+      numSplits = cli.getOptionValue("n");
+    }
+
+    if (cli.getArgs().length > 0) {
+      query = cli.getArgs()[0];
+    }
+
+    System.out.println("url: "+url);
+    System.out.println("user: "+user);
+    System.out.println("query: "+query);
+
+    LlapBaseInputFormat format = new LlapBaseInputFormat(url, user, pwd, query);
+    JobConf job = new JobConf();
+
+    InputSplit[] splits = format.getSplits(job, Integer.parseInt(numSplits));
+
+    if (splits.length == 0) {
+      System.out.println("No splits returned - empty scan");
+      System.out.println("Results: ");
+    } else {
+      boolean first = true;
+
+      for (InputSplit s: splits) {
+        LOG.info("Processing input split s from " + Arrays.toString(s.getLocations()));
+        RecordReader<NullWritable, Text> reader = format.getRecordReader(s, job, null);
+
+        if (reader instanceof LlapBaseRecordReader && first) {
+          Schema schema = ((LlapBaseRecordReader)reader).getSchema();
+          System.out.println(""+schema);
+        }
+
+        if (first) {
+          System.out.println("Results: ");
+          System.out.println("");
+          first = false;
+        }
+
+        Text value = reader.createValue();
+        while (reader.next(NullWritable.get(), value)) {
+          System.out.println(value);
+        }
+      }
+      System.exit(0);
+    }
+  }
+
+  static Options createOptions() {
+    Options result = new Options();
+
+    result.addOption(OptionBuilder
+        .withLongOpt("location")
+        .withDescription("HS2 url")
+        .hasArg()
+        .create('l'));
+
+    result.addOption(OptionBuilder
+        .withLongOpt("user")
+        .withDescription("user name")
+        .hasArg()
+        .create('u'));
+
+    result.addOption(OptionBuilder
+        .withLongOpt("pwd")
+        .withDescription("password")
+        .hasArg()
+        .create('p'));
+
+    result.addOption(OptionBuilder
+        .withLongOpt("num")
+        .withDescription("number of splits")
+        .hasArg()
+        .create('n'));
+
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
new file mode 100644
index 0000000..6ecb0f9
--- /dev/null
+++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
@@ -0,0 +1,36 @@
+package org.apache.hadoop.hive.llap;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hive.llap.LlapBaseRecordReader;
+import org.apache.hadoop.hive.llap.LlapRowRecordReader;
+import org.apache.hadoop.hive.llap.Row;
+import org.apache.hadoop.hive.llap.Schema;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+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.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hive.llap.ext.LlapInputSplit;
+
+
+public class LlapRowInputFormat implements InputFormat<NullWritable, Row> {
+  LlapBaseInputFormat<Text> baseInputFormat = new LlapBaseInputFormat<Text>();
+
+  @Override
+  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
+    return baseInputFormat.getSplits(job, numSplits);
+  }
+
+  @Override
+  public RecordReader<NullWritable, Row> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
+      throws IOException {
+    LlapInputSplit<Text> llapSplit = (LlapInputSplit<Text>) split;
+    LlapBaseRecordReader<Text> reader = (LlapBaseRecordReader<Text>) baseInputFormat.getRecordReader(llapSplit, job, reporter);
+    return new LlapRowRecordReader(job, reader.getSchema(), reader);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/llap-ext-client/src/java/org/apache/hive/llap/ext/LlapInputSplit.java
----------------------------------------------------------------------
diff --git a/llap-ext-client/src/java/org/apache/hive/llap/ext/LlapInputSplit.java b/llap-ext-client/src/java/org/apache/hive/llap/ext/LlapInputSplit.java
new file mode 100644
index 0000000..d8881c4
--- /dev/null
+++ b/llap-ext-client/src/java/org/apache/hive/llap/ext/LlapInputSplit.java
@@ -0,0 +1,73 @@
+package org.apache.hive.llap.ext;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
+import org.apache.hadoop.mapred.SplitLocationInfo;
+
+
+public class LlapInputSplit<V extends WritableComparable> implements InputSplitWithLocationInfo {
+  InputSplitWithLocationInfo nativeSplit;
+  String inputFormatClassName;
+
+  public LlapInputSplit() {
+  }
+
+  public LlapInputSplit(InputSplitWithLocationInfo nativeSplit, String inputFormatClassName) {
+    this.nativeSplit = nativeSplit;
+    this.inputFormatClassName = inputFormatClassName;
+  }
+
+  @Override
+  public long getLength() throws IOException {
+    return nativeSplit.getLength();
+  }
+
+  @Override
+  public String[] getLocations() throws IOException {
+    return nativeSplit.getLocations();
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeUTF(inputFormatClassName);
+    out.writeUTF(nativeSplit.getClass().getName());
+    nativeSplit.write(out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    inputFormatClassName = in.readUTF();
+    String splitClass = in.readUTF();
+    try {
+      nativeSplit = (InputSplitWithLocationInfo)Class.forName(splitClass).newInstance();
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+    nativeSplit.readFields(in);
+  }
+
+  @Override
+  public SplitLocationInfo[] getLocationInfo() throws IOException {
+    return nativeSplit.getLocationInfo();
+  }
+
+  public InputSplit getSplit() {
+    return nativeSplit;
+  }
+
+  public InputFormat<NullWritable, V> getInputFormat() throws IOException {
+    try {
+      return (InputFormat<NullWritable, V>) Class.forName(inputFormatClassName)
+          .newInstance();
+    } catch(Exception e) {
+      throw new IOException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
deleted file mode 100644
index 0930d60..0000000
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
+++ /dev/null
@@ -1,392 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.llap;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import com.google.common.collect.Lists;
-import com.google.protobuf.ByteString;
-
-import org.apache.commons.collections4.ListUtils;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.llap.LlapBaseRecordReader.ReaderEvent;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
-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.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.token.Token;
-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.runtime.api.events.TaskAttemptFailedEvent;
-import org.apache.tez.runtime.api.impl.TaskSpec;
-import org.apache.tez.runtime.api.impl.TezEvent;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.tez.dag.records.TezTaskAttemptID;
-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;
-
-
-public class LlapInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
-
-  private static final Logger LOG = LoggerFactory.getLogger(LlapInputFormat.class);
-
-  public LlapInputFormat() {
-  }
-
-  /*
-   * This proxy record reader has the duty of establishing a connected socket with LLAP, then fire
-   * off the work in the split to LLAP and finally return the connected socket back in an
-   * LlapRecordReader. The LlapRecordReader class reads the results from the socket.
-   */
-  @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 {
-    throw new IOException("These are not the splits you are looking for.");
-  }
-
-  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();
-    // 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());
-
-    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);
-
-
-    builder.setUsingTezAm(false);
-    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/7b9096a9/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java b/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
deleted file mode 100644
index 7d06637..0000000
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
+++ /dev/null
@@ -1,415 +0,0 @@
-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.SubmitWorkRequestProto;
-import org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol;
-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 {
-    int numHandlers = HiveConf.getIntVar(conf,
-        HiveConf.ConfVars.LLAP_TMP_EXT_CLIENT_NUM_SERVER_HANDLERS);
-    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. This should always have the usingTezAm flag disabled
-   * @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(
-        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.";
-                LOG.info(msg);
-                if (responder != null) {
-                  Throwable err = new RuntimeException(msg);
-                  responder.submissionFailed(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), err);
-                }
-                return;
-              }
-            }
-          }
-
-          @Override
-          public void indicateError(Throwable t) {
-            String msg = "Failed to submit: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString();
-            LOG.error(msg, t);
-            Throwable err = new RuntimeException(msg, t);
-            responder.submissionFailed(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), 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/7b9096a9/llap-server/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
deleted file mode 100644
index dbd591a..0000000
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
+++ /dev/null
@@ -1,57 +0,0 @@
-package org.apache.hadoop.hive.llap.tezplugins.helpers;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.token.Token;
-import org.apache.tez.common.security.JobTokenIdentifier;
-import org.apache.tez.common.security.JobTokenSecretManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class LlapTaskUmbilicalServer {
-
-  private static final Logger LOG = LoggerFactory.getLogger(LlapTaskUmbilicalServer.class);
-
-  protected volatile Server server;
-  private final InetSocketAddress address;
-  private final AtomicBoolean started = new AtomicBoolean(true);
-
-  public LlapTaskUmbilicalServer(Configuration conf, LlapTaskUmbilicalProtocol umbilical, int numHandlers, String tokenIdentifier, Token<JobTokenIdentifier> token) throws
-      IOException {
-    JobTokenSecretManager jobTokenSecretManager =
-        new JobTokenSecretManager();
-    jobTokenSecretManager.addTokenForJob(tokenIdentifier, token);
-
-    server = new RPC.Builder(conf)
-        .setProtocol(LlapTaskUmbilicalProtocol.class)
-        .setBindAddress("0.0.0.0")
-        .setPort(0)
-        .setInstance(umbilical)
-        .setNumHandlers(numHandlers)
-        .setSecretManager(jobTokenSecretManager).build();
-
-    server.start();
-    this.address = NetUtils.getConnectAddress(server);
-    LOG.info(
-        "Started TaskUmbilicalServer: " + umbilical.getClass().getName() + " at address: " + address +
-            " with numHandlers=" + numHandlers);
-  }
-
-  public InetSocketAddress getAddress() {
-    return this.address;
-  }
-
-  public void shutdownServer() {
-    if (started.get()) { // Primarily to avoid multiple shutdowns.
-      started.set(false);
-      server.stop();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2337e89..f773d2f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -50,6 +50,7 @@
     <module>service</module>
     <module>llap-common</module>
     <module>llap-client</module>
+    <module>llap-ext-client</module>
     <module>llap-tez</module>
     <module>llap-server</module>
     <module>shims</module>

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/ql/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
deleted file mode 100644
index 7073280..0000000
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
+++ /dev/null
@@ -1,205 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.llap;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.DataInputStream;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.llap.Schema;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.JobConf;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class LlapBaseRecordReader<V extends WritableComparable> implements RecordReader<NullWritable, V> {
-  private static final Logger LOG = LoggerFactory.getLogger(LlapBaseRecordReader.class);
-
-  DataInputStream din;
-  Schema schema;
-  Class<V> clazz;
-
-
-  protected Thread readerThread = null;
-  protected LinkedBlockingQueue<ReaderEvent> readerEvents = new LinkedBlockingQueue<ReaderEvent>();
-
-  public LlapBaseRecordReader(InputStream in, Schema schema, Class<V> clazz) {
-    din = new DataInputStream(in);
-    this.schema = schema;
-    this.clazz = clazz;
-    this.readerThread = Thread.currentThread();
-  }
-
-  public Schema getSchema() {
-    return schema;
-  }
-
-  @Override
-  public void close() throws IOException {
-    din.close();
-  }
-
-  @Override
-  public long getPos() { return 0; }
-
-  @Override
-  public float getProgress() { return 0f; }
-
-  @Override
-  public NullWritable createKey() {
-    return NullWritable.get();
-  }
-
-  @Override
-  public V createValue() {
-    try {
-      return clazz.newInstance();
-    } catch (Exception e) {
-      return null;
-    }
-  }
-
-  @Override
-  public boolean next(NullWritable key, V value) throws IOException {
-    try {
-      // Need a way to know what thread to interrupt, since this is a blocking thread.
-      setReaderThread(Thread.currentThread());
-
-      value.readFields(din);
-      return true;
-    } catch (EOFException eof) {
-      // End of input. There should be a reader event available, or coming soon, so okay to be blocking call.
-      ReaderEvent event = getReaderEvent();
-      switch (event.getEventType()) {
-        case DONE:
-          break;
-        default:
-          throw new IOException("Expected reader event with done status, but got "
-              + event.getEventType() + " with message " + event.getMessage());
-      }
-      return false;
-    } catch (IOException io) {
-      if (Thread.interrupted()) {
-        // Either we were interrupted by one of:
-        // 1. handleEvent(), in which case there is a reader event waiting for us in the queue
-        // 2. Some other unrelated cause which interrupted us, in which case there may not be a reader event coming.
-        // Either way we should not try to block trying to read the reader events queue.
-        if (readerEvents.isEmpty()) {
-          // Case 2.
-          throw io;
-        } else {
-          // Case 1. Fail the reader, sending back the error we received from the reader event.
-          ReaderEvent event = getReaderEvent();
-          switch (event.getEventType()) {
-            case ERROR:
-              throw new IOException("Received reader event error: " + event.getMessage());
-            default:
-              throw new IOException("Got reader event type " + event.getEventType() + ", expected error event");
-          }
-        }
-      } else {
-        // If we weren't interrupted, just propagate the error
-        throw io;
-      }
-    }
-  }
-
-  /**
-   * Define success/error events which are passed to the reader from a different thread.
-   * The reader will check for these events on end of input and interruption of the reader thread.
-   */
-  public static class ReaderEvent {
-    public enum EventType {
-      DONE,
-      ERROR
-    }
-
-    protected final EventType eventType;
-    protected final String message;
-
-    protected ReaderEvent(EventType type, String message) {
-      this.eventType = type;
-      this.message = message;
-    }
-
-    public static ReaderEvent doneEvent() {
-      return new ReaderEvent(EventType.DONE, "");
-    }
-
-    public static ReaderEvent errorEvent(String message) {
-      return new ReaderEvent(EventType.ERROR, message);
-    }
-
-    public EventType getEventType() {
-      return eventType;
-    }
-
-    public String getMessage() {
-      return message;
-    }
-  }
-
-  public void handleEvent(ReaderEvent event) {
-    switch (event.getEventType()) {
-      case DONE:
-        // Reader will check for the event queue upon the end of the input stream - no need to interrupt.
-        readerEvents.add(event);
-        break;
-      case ERROR:
-        readerEvents.add(event);
-        if (readerThread == null) {
-          throw new RuntimeException("Reader thread is unexpectedly null, during ReaderEvent error " + event.getMessage());
-        }
-        // Reader is using a blocking socket .. interrupt it.
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Interrupting reader thread due to reader event with error " + event.getMessage());
-        }
-        getReaderThread().interrupt();
-        break;
-      default:
-        throw new RuntimeException("Unhandled ReaderEvent type " + event.getEventType() + " with message " + event.getMessage());
-    }
-  }
-
-  protected ReaderEvent getReaderEvent() {
-    try {
-      ReaderEvent event = readerEvents.take();
-      return event;
-    } catch (InterruptedException ie) {
-      throw new RuntimeException("Interrupted while getting readerEvents, not expected", ie);
-    }
-  }
-
-  protected synchronized void setReaderThread(Thread readerThread) {
-    this.readerThread = readerThread;
-  }
-
-  protected synchronized Thread getReaderThread() {
-    return readerThread;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
deleted file mode 100644
index 02aedfd..0000000
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.llap;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.hive.llap.Schema;
-import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
-import org.apache.hadoop.mapred.SplitLocationInfo;
-import org.apache.thrift.TDeserializer;
-import org.apache.thrift.TSerializer;
-
-public class LlapInputSplit implements InputSplitWithLocationInfo {
-
-  int splitNum;
-  byte[] planBytes;
-  byte[] fragmentBytes;
-  SplitLocationInfo[] locations;
-  Schema schema;
-  String llapUser;
-
-  public LlapInputSplit() {
-  }
-
-  public LlapInputSplit(int splitNum, byte[] planBytes, byte[] fragmentBytes, SplitLocationInfo[] locations, Schema schema, String llapUser) {
-    this.planBytes = planBytes;
-    this.fragmentBytes = fragmentBytes;
-    this.locations = locations;
-    this.schema = schema;
-    this.splitNum = splitNum;
-    this.llapUser = llapUser;
-  }
-
-  public Schema getSchema() {
-    return schema;
-  }
-
-  @Override
-  public long getLength() throws IOException {
-    return 0;
-  }
-
-  @Override
-  public String[] getLocations() throws IOException {
-    String[] locs = new String[locations.length];
-    for (int i = 0; i < locations.length; ++i) {
-      locs[i] = locations[i].getLocation();
-    }
-    return locs;
-  }
-
-  public int getSplitNum() {
-    return splitNum;
-  }
-
-  public byte[] getPlanBytes() {
-    return planBytes;
-  }
-
-  public byte[] getFragmentBytes() {
-    return fragmentBytes;
-  }
-
-
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(splitNum);
-    out.writeInt(planBytes.length);
-    out.write(planBytes);
-
-    out.writeInt(fragmentBytes.length);
-    out.write(fragmentBytes);
-
-    out.writeInt(locations.length);
-    for (int i = 0; i < locations.length; ++i) {
-      out.writeUTF(locations[i].getLocation());
-    }
-
-    schema.write(out);
-    out.writeUTF(llapUser);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    splitNum = in.readInt();
-    int length = in.readInt();
-    planBytes = new byte[length];
-    in.readFully(planBytes);
-
-    length = in.readInt();
-    fragmentBytes = new byte[length];
-    in.readFully(fragmentBytes);
-
-    length = in.readInt();
-    locations = new SplitLocationInfo[length];
-
-    for (int i = 0; i < length; ++i) {
-      locations[i] = new SplitLocationInfo(in.readUTF(), false);
-    }
-
-    schema = new Schema();
-    schema.readFields(in);
-    llapUser = in.readUTF();
-  }
-
-  @Override
-  public SplitLocationInfo[] getLocationInfo() throws IOException {
-    return locations;
-  }
-
-  public String getLlapUser() {
-    return llapUser;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b9096a9/ql/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java b/ql/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
deleted file mode 100644
index 83149ab..0000000
--- a/ql/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
+++ /dev/null
@@ -1,103 +0,0 @@
-package org.apache.hadoop.hive.llap;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.common.security.JobTokenIdentifier;
-import org.apache.tez.common.security.JobTokenSecretManager;
-import org.apache.tez.runtime.api.impl.TaskSpec;
-
-public class SubmitWorkInfo implements Writable {
-
-  private TaskSpec taskSpec;
-  private ApplicationId fakeAppId;
-  private long creationTime;
-
-  // This is used to communicate over the LlapUmbilicalProtocol. Not related to tokens used to
-  // talk to LLAP daemons itself via the securit work.
-  private Token<JobTokenIdentifier> token;
-
-  public SubmitWorkInfo(TaskSpec taskSpec, ApplicationId fakeAppId, long creationTime) {
-    this.taskSpec = taskSpec;
-    this.fakeAppId = fakeAppId;
-    this.token = createJobToken();
-    this.creationTime = creationTime;
-  }
-
-  // Empty constructor for writable etc.
-  public SubmitWorkInfo() {
-  }
-
-  public TaskSpec getTaskSpec() {
-    return taskSpec;
-  }
-
-  public ApplicationId getFakeAppId() {
-    return fakeAppId;
-  }
-
-  public String getTokenIdentifier() {
-    return fakeAppId.toString();
-  }
-
-  public Token<JobTokenIdentifier> getToken() {
-    return token;
-  }
-
-  public long getCreationTime() {
-    return creationTime;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    taskSpec.write(out);
-    out.writeLong(fakeAppId.getClusterTimestamp());
-    out.writeInt(fakeAppId.getId());
-    token.write(out);
-    out.writeLong(creationTime);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    taskSpec = new TaskSpec();
-    taskSpec.readFields(in);
-    long appIdTs = in.readLong();
-    int appIdId = in.readInt();
-    fakeAppId = ApplicationId.newInstance(appIdTs, appIdId);
-    token = new Token<>();
-    token.readFields(in);
-    creationTime = in.readLong();
-  }
-
-  public static byte[] toBytes(SubmitWorkInfo submitWorkInfo) throws IOException {
-    DataOutputBuffer dob = new DataOutputBuffer();
-    submitWorkInfo.write(dob);
-    return dob.getData();
-  }
-
-  public static SubmitWorkInfo fromBytes(byte[] submitWorkInfoBytes) throws IOException {
-    DataInputBuffer dib = new DataInputBuffer();
-    dib.reset(submitWorkInfoBytes, 0, submitWorkInfoBytes.length);
-    SubmitWorkInfo submitWorkInfo = new SubmitWorkInfo();
-    submitWorkInfo.readFields(dib);
-    return submitWorkInfo;
-  }
-
-
-  private Token<JobTokenIdentifier> createJobToken() {
-    String tokenIdentifier = fakeAppId.toString();
-    JobTokenIdentifier identifier = new JobTokenIdentifier(new Text(
-        tokenIdentifier));
-    Token<JobTokenIdentifier> sessionToken = new Token<JobTokenIdentifier>(identifier,
-        new JobTokenSecretManager());
-    sessionToken.setService(identifier.getJobId());
-    return sessionToken;
-  }
-}


[37/39] hive git commit: Merge branch 'master' into llap

Posted by jd...@apache.org.
Merge branch 'master' into llap


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/763e6969
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/763e6969
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/763e6969

Branch: refs/heads/master
Commit: 763e6969d0e78806db0fc875830395c783f18b0c
Parents: 03ee048 0927187
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu May 5 13:03:53 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu May 5 13:03:53 2016 -0700

----------------------------------------------------------------------
 .../src/main/resources/hive-log4j2.properties   |    6 +-
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     |  108 +-
 .../main/java/org/apache/hive/hplsql/Exec.java  |   67 +-
 .../java/org/apache/hive/hplsql/Expression.java |   31 +-
 .../java/org/apache/hive/hplsql/Select.java     |   31 +-
 .../java/org/apache/hive/hplsql/Signal.java     |    2 +-
 .../main/java/org/apache/hive/hplsql/Stmt.java  |  154 +-
 hplsql/src/main/resources/hplsql-site.xml       |    2 -
 .../org/apache/hive/hplsql/TestHplsqlLocal.java |    5 +
 .../apache/hive/hplsql/TestHplsqlOffline.java   |   20 +
 hplsql/src/test/queries/local/if3_bteq.sql      |    3 +
 .../test/queries/offline/create_table_td.sql    |   45 +
 hplsql/src/test/queries/offline/delete_all.sql  |    1 +
 hplsql/src/test/queries/offline/select.sql      |   42 +
 .../test/queries/offline/select_teradata.sql    |   12 +
 hplsql/src/test/results/db/select_into.out.txt  |    3 +-
 hplsql/src/test/results/db/select_into2.out.txt |    4 +-
 hplsql/src/test/results/local/if3_bteq.out.txt  |    3 +
 hplsql/src/test/results/local/lang.out.txt      |   10 +-
 .../results/offline/create_table_mssql.out.txt  |   39 +-
 .../results/offline/create_table_mssql2.out.txt |   13 +-
 .../results/offline/create_table_mysql.out.txt  |    5 +-
 .../results/offline/create_table_ora.out.txt    |   65 +-
 .../results/offline/create_table_ora2.out.txt   |    9 +-
 .../results/offline/create_table_pg.out.txt     |    7 +-
 .../results/offline/create_table_td.out.txt     |   31 +
 .../src/test/results/offline/delete_all.out.txt |    2 +
 hplsql/src/test/results/offline/select.out.txt  |   34 +
 .../src/test/results/offline/select_db2.out.txt |    3 +-
 .../results/offline/select_teradata.out.txt     |   10 +
 .../hadoop/hive/llap/cache/BuddyAllocator.java  |   43 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |    5 +-
 metastore/if/hive_metastore.thrift              |    8 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2431 ++++++++++--------
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  133 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 2180 ++++++++--------
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   52 +
 .../metastore/api/DropConstraintRequest.java    |  591 +++++
 .../hive/metastore/api/ThriftHiveMetastore.java | 1966 ++++++++++----
 .../gen-php/metastore/ThriftHiveMetastore.php   |  242 ++
 .../src/gen/thrift/gen-php/metastore/Types.php  |  121 +
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       |  212 ++
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   97 +
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   23 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   63 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   29 +
 .../hive/metastore/HiveMetaStoreClient.java     |    6 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |    3 +
 .../hadoop/hive/metastore/ObjectStore.java      |   46 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |    2 +
 .../hive/metastore/RetryingMetaStoreClient.java |   17 +-
 .../hadoop/hive/metastore/hbase/HBaseStore.java |    6 +
 .../DummyRawStoreControlledCommit.java          |    6 +
 .../DummyRawStoreForJdoConnection.java          |    6 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   21 +-
 .../persistence/HybridHashTableContainer.java   |   60 +-
 .../ql/exec/persistence/KeyValueContainer.java  |    4 +
 .../ql/exec/vector/VectorizationContext.java    |    7 +
 .../hadoop/hive/ql/hooks/WriteEntity.java       |    3 +-
 .../serde/AbstractParquetMapInspector.java      |    4 +-
 .../serde/ParquetHiveArrayInspector.java        |    4 +-
 .../ql/io/parquet/write/DataWritableWriter.java |   67 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   12 +-
 .../rules/HiveReduceExpressionsRule.java        |  125 +
 .../rules/HiveSortLimitPullUpConstantsRule.java |  157 ++
 .../rules/HiveUnionPullUpConstantsRule.java     |  133 +
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |    5 +
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   13 +-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |    9 +
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |    2 +
 .../hadoop/hive/ql/plan/AlterTableDesc.java     |   25 +-
 .../hadoop/hive/ql/plan/HiveOperation.java      |    2 +
 .../ql/io/parquet/TestDataWritableWriter.java   |   29 +
 .../serde/TestAbstractParquetMapInspector.java  |    4 +-
 .../serde/TestParquetHiveArrayInspector.java    |    4 +-
 .../clientnegative/drop_invalid_constraint1.q   |    3 +
 .../clientnegative/drop_invalid_constraint2.q   |    2 +
 .../clientnegative/drop_invalid_constraint3.q   |    2 +
 .../clientnegative/drop_invalid_constraint4.q   |    3 +
 .../test/queries/clientpositive/cbo_input26.q   |   54 +
 .../queries/clientpositive/cbo_union_view.q     |   19 +
 .../clientpositive/create_with_constraints.q    |   12 +
 .../parquet_array_map_emptynullvals.q           |   20 +
 .../vector_non_constant_in_expr.q               |    4 +
 .../drop_invalid_constraint1.q.out              |   15 +
 .../drop_invalid_constraint2.q.out              |   11 +
 .../drop_invalid_constraint3.q.out              |   11 +
 .../drop_invalid_constraint4.q.out              |   19 +
 .../results/clientpositive/cbo_input26.q.out    |  596 +++++
 .../results/clientpositive/cbo_union_view.q.out |  228 ++
 .../create_with_constraints.q.out               |   68 +
 .../results/clientpositive/groupby_ppd.q.out    |   28 +-
 .../clientpositive/load_dyn_part14.q.out        |    6 +-
 .../parquet_array_map_emptynullvals.q.out       |   87 +
 .../results/clientpositive/perf/query66.q.out   |  328 +--
 .../results/clientpositive/perf/query75.q.out   |  692 ++---
 .../clientpositive/spark/load_dyn_part14.q.out  |    6 +-
 .../clientpositive/spark/union_remove_25.q.out  |   16 +-
 .../clientpositive/spark/union_view.q.out       |   60 +-
 .../clientpositive/union_remove_25.q.out        |   20 +-
 .../results/clientpositive/union_view.q.out     |   60 +-
 .../vector_non_constant_in_expr.q.out           |   36 +
 service/src/gen/thrift/gen-py/__init__.py       |    0
 105 files changed, 8771 insertions(+), 3392 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/763e6969/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------


[29/39] 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/33c86c45
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/33c86c45
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/33c86c45

Branch: refs/heads/master
Commit: 33c86c45c604d32c9f9719c30f700732402e9977
Parents: 6d1b6bb
Author: Jason Dere <jd...@hortonworks.com>
Authored: Mon May 2 17:51:57 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Mon May 2 17:51:57 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/33c86c45/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/33c86c45/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/33c86c45/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/33c86c45/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/33c86c45/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());


[30/39] hive git commit: reverting HIVE-13674: usingTezAm field not required in LLAP SubmitWorkRequestProto

Posted by jd...@apache.org.
reverting 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/342668f9
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/342668f9
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/342668f9

Branch: refs/heads/master
Commit: 342668f914587aa396a7f74c421c4dcf2037d433
Parents: 33c86c4
Author: Jason Dere <jd...@hortonworks.com>
Authored: Mon May 2 18:23:30 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Mon May 2 18:23:30 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, 175 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/342668f9/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 fe2fd7c..8598bc8 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,10 +123,12 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
 
 
   /**
-   * Submit the work for actual execution.
+   * Submit the work for actual execution. This should always have the usingTezAm flag disabled
    * @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/342668f9/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 6a20031..653e7e0 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: llap-common/src/protobuf/LlapDaemonProtocol.proto
+// source: LlapDaemonProtocol.proto
 
 package org.apache.hadoop.hive.llap.daemon.rpc;
 
@@ -7334,6 +7334,16 @@ 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}
@@ -7452,6 +7462,11 @@ public final class LlapDaemonProtocolProtos {
               bitField0_ |= 0x00000200;
               break;
             }
+            case 88: {
+              bitField0_ |= 0x00000400;
+              usingTezAm_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -7799,6 +7814,22 @@ 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_ = "";
@@ -7810,6 +7841,7 @@ 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() {
@@ -7853,6 +7885,9 @@ public final class LlapDaemonProtocolProtos {
       if (((bitField0_ & 0x00000200) == 0x00000200)) {
         output.writeMessage(10, fragmentRuntimeInfo_);
       }
+      if (((bitField0_ & 0x00000400) == 0x00000400)) {
+        output.writeBool(11, usingTezAm_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -7902,6 +7937,10 @@ 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;
@@ -7975,6 +8014,11 @@ 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;
@@ -8028,6 +8072,10 @@ 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;
@@ -8167,6 +8215,8 @@ public final class LlapDaemonProtocolProtos {
           fragmentRuntimeInfoBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000200);
+        usingTezAm_ = true;
+        bitField0_ = (bitField0_ & ~0x00000400);
         return this;
       }
 
@@ -8243,6 +8293,10 @@ 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;
@@ -8299,6 +8353,9 @@ public final class LlapDaemonProtocolProtos {
         if (other.hasFragmentRuntimeInfo()) {
           mergeFragmentRuntimeInfo(other.getFragmentRuntimeInfo());
         }
+        if (other.hasUsingTezAm()) {
+          setUsingTezAm(other.getUsingTezAm());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -9032,6 +9089,39 @@ 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)
     }
 
@@ -14365,74 +14455,74 @@ public final class LlapDaemonProtocolProtos {
       descriptor;
   static {
     java.lang.String[] descriptorData = {
-      "\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"
+      "\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"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -14486,7 +14576,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", });
+              new java.lang.String[] { "ContainerIdString", "AmHost", "AmPort", "TokenIdentifier", "CredentialsBinary", "User", "ApplicationIdString", "AppAttemptNumber", "FragmentSpec", "FragmentRuntimeInfo", "UsingTezAm", });
           internal_static_SubmitWorkResponseProto_descriptor =
             getDescriptor().getMessageTypes().get(8);
           internal_static_SubmitWorkResponseProto_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hive/blob/342668f9/llap-common/src/protobuf/LlapDaemonProtocol.proto
----------------------------------------------------------------------
diff --git a/llap-common/src/protobuf/LlapDaemonProtocol.proto b/llap-common/src/protobuf/LlapDaemonProtocol.proto
index 944c96c..e964c5f 100644
--- a/llap-common/src/protobuf/LlapDaemonProtocol.proto
+++ b/llap-common/src/protobuf/LlapDaemonProtocol.proto
@@ -91,6 +91,7 @@ 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 {
@@ -136,11 +137,18 @@ 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/342668f9/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 8db2f88..10d14c0 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,6 +346,7 @@ 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/342668f9/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 4a33373..efd6f0a 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,6 +108,7 @@ 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);
@@ -136,6 +137,8 @@ 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());


[15/39] hive git commit: HIVE-HIVE-13419 part 2. Merge remote-tracking branch 'origin/master' into llap

Posted by jd...@apache.org.
HIVE-HIVE-13419 part 2. Merge remote-tracking branch 'origin/master' into llap

Conflicts:
	llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/79c1c691
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/79c1c691
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/79c1c691

Branch: refs/heads/master
Commit: 79c1c691e529107cba27eabefb94707645a39987
Parents: a7b0ca7 91ab819
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Apr 4 15:37:10 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Apr 4 15:37:10 2016 -0700

----------------------------------------------------------------------
 beeline/pom.xml.orig                            |  169 +
 .../java/org/apache/hive/beeline/BeeLine.java   |   18 +-
 .../apache/hive/beeline/cli/TestHiveCli.java    |   15 +-
 data/conf/llap/hive-site.xml                    |   44 +
 data/conf/llap/llap-daemon-site.xml             |   61 -
 .../org/apache/hive/minikdc/MiniHiveKdc.java    |   46 +-
 .../minikdc/TestJdbcNonKrbSASLWithMiniKdc.java  |  103 +
 itests/hive-unit/pom.xml                        |   13 +-
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |   26 +-
 .../test/resources/testconfiguration.properties |    7 -
 itests/util/pom.xml                             |   11 +
 .../apache/hadoop/hive/llap/LlapItUtils.java    |   90 +
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |   25 +-
 .../impl/LlapZookeeperRegistryImpl.java         |    9 +-
 .../hive/llap/tez/LlapProtocolClientProxy.java  |    4 +-
 .../hive/llap/cli/LlapOptionsProcessor.java     |   25 +-
 .../configuration/LlapDaemonConfiguration.java  |    8 +
 .../hive/llap/daemon/impl/LlapDaemon.java       |   34 +-
 .../daemon/impl/LlapProtocolServerImpl.java     |    7 +-
 .../daemon/services/impl/LlapWebServices.java   |   13 +-
 .../hive/llap/daemon/MiniLlapCluster.java       |  147 +-
 .../llap/tezplugins/LlapTaskCommunicator.java   |   37 +-
 .../tezplugins/LlapTaskSchedulerService.java    |    7 +
 .../tezplugins/helpers/SourceStateTracker.java  |    2 +-
 .../tezplugins/TestLlapTaskCommunicator.java    |  304 +-
 .../java/org/apache/orc/impl/IntegerReader.java |    4 +-
 .../apache/orc/impl/RunLengthIntegerReader.java |    7 +-
 .../orc/impl/RunLengthIntegerReaderV2.java      |    7 +-
 .../org/apache/orc/impl/SerializationUtils.java |   34 +-
 .../apache/orc/impl/TestSerializationUtils.java |   45 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |    9 +-
 .../ql/exec/spark/HiveSparkClientFactory.java   |    9 +
 .../ql/exec/spark/RemoteHiveSparkClient.java    |   20 +-
 .../hadoop/hive/ql/exec/spark/SparkTask.java    |    3 +
 .../exec/spark/status/LocalSparkJobMonitor.java |    2 +-
 .../spark/status/RemoteSparkJobMonitor.java     |    5 +-
 .../ql/exec/vector/VectorizationContext.java    |   12 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |   16 +-
 .../hive/ql/io/orc/TreeReaderFactory.java       |   46 +-
 .../hadoop/hive/ql/parse/ParseContext.java      |    7 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   29 +-
 .../hadoop/hive/ql/parse/TaskCompiler.java      |    2 +-
 .../hive/ql/plan/ExprNodeGenericFuncDesc.java   |    5 +-
 .../hive/ql/exec/TestFunctionRegistry.java      |   18 +-
 .../exec/vector/TestVectorizationContext.java   |   17 +-
 .../queries/clientpositive/cast_on_constant.q   |    7 +
 ql/src/test/queries/clientpositive/macro_1.q    |   29 +
 .../clientpositive/cast_on_constant.q.out       |  160 +
 .../test/results/clientpositive/macro_1.q.out   |   84 +
 .../clientpositive/spark/temp_table.q.out       |    2 +
 service-rpc/src/gen/thrift/gen-py/__init__.py   |    0
 service/src/gen/thrift/gen-cpp/ThriftHive.cpp   | 3544 ++++++++
 service/src/gen/thrift/gen-cpp/ThriftHive.h     | 1224 +++
 .../gen-cpp/ThriftHive_server.skeleton.cpp      |   84 +
 .../thrift/gen-cpp/hive_service_constants.cpp   |   17 +
 .../gen/thrift/gen-cpp/hive_service_constants.h |   24 +
 .../gen/thrift/gen-cpp/hive_service_types.cpp   |  351 +
 .../src/gen/thrift/gen-cpp/hive_service_types.h |  176 +
 .../hadoop/hive/service/HiveClusterStatus.java  |  901 ++
 .../hive/service/HiveServerException.java       |  601 ++
 .../hadoop/hive/service/JobTrackerState.java    |   45 +
 .../apache/hadoop/hive/service/ThriftHive.java  | 7784 ++++++++++++++++++
 service/src/gen/thrift/gen-php/ThriftHive.php   | 1943 +++++
 service/src/gen/thrift/gen-php/Types.php        |  338 +
 service/src/gen/thrift/gen-py/__init__.py       |    0
 .../gen-py/hive_service/ThriftHive-remote       | 1242 +++
 .../thrift/gen-py/hive_service/ThriftHive.py    | 1674 ++++
 .../gen/thrift/gen-py/hive_service/__init__.py  |    1 +
 .../gen/thrift/gen-py/hive_service/constants.py |   11 +
 .../gen/thrift/gen-py/hive_service/ttypes.py    |  260 +
 .../gen/thrift/gen-rb/hive_service_constants.rb |    9 +
 .../src/gen/thrift/gen-rb/hive_service_types.rb |   68 +
 service/src/gen/thrift/gen-rb/thrift_hive.rb    |  555 ++
 .../hive/service/auth/HiveAuthFactory.java      |   13 +-
 .../service/cli/thrift/ThriftCLIService.java    |   15 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |   66 +-
 .../apache/hadoop/hive/shims/HadoopShims.java   |    2 +-
 .../hive/spark/client/SparkClientImpl.java      |    9 +-
 78 files changed, 22374 insertions(+), 387 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/79c1c691/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/79c1c691/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/79c1c691/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/79c1c691/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --cc llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index e9213ee,2fe59a2..418d87a
--- 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
@@@ -290,9 -292,23 +293,24 @@@ public class LlapDaemon extends Composi
      ShuffleHandler.initializeAndStart(shuffleHandlerConf);
      LOG.info("Setting shuffle port to: " + ShuffleHandler.get().getPort());
      this.shufflePort.set(ShuffleHandler.get().getPort());
+     getConfig()
+         .setInt(ConfVars.LLAP_DAEMON_YARN_SHUFFLE_PORT.varname, ShuffleHandler.get().getPort());
      super.serviceStart();
+ 
+     // Setup the actual ports in the configuration.
+     getConfig().setInt(ConfVars.LLAP_DAEMON_RPC_PORT.varname, server.getBindAddress().getPort());
+     getConfig().setInt(ConfVars.LLAP_MANAGEMENT_RPC_PORT.varname, server.getManagementBindAddress().getPort());
+     if (webServices != null) {
+       getConfig().setInt(ConfVars.LLAP_DAEMON_WEB_PORT.varname, webServices.getPort());
+     }
 +    LlapOutputFormatService.get();
-     LOG.info("LlapDaemon serviceStart complete");
+ 
+     this.registry.init(getConfig());
+     this.registry.start();
+     LOG.info(
+         "LlapDaemon serviceStart complete. RPC Port={}, ManagementPort={}, ShuflePort={}, WebPort={}",
+         server.getBindAddress().getPort(), server.getManagementBindAddress().getPort(),
+         ShuffleHandler.get().getPort(), (webServices == null ? "" : webServices.getPort()));
    }
  
    public void serviceStop() throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/79c1c691/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/79c1c691/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/79c1c691/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------


[35/39] hive git commit: HIVE-13620: Merge llap branch work to master (committing changes from review feedback)

Posted by jd...@apache.org.
HIVE-13620: Merge llap branch work to master (committing changes from review feedback)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2a03f1f4
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2a03f1f4
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2a03f1f4

Branch: refs/heads/master
Commit: 2a03f1f4648c683414c0b23be0aebbfd614d105c
Parents: e057909
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu May 5 12:29:14 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu May 5 12:29:14 2016 -0700

----------------------------------------------------------------------
 .../hive/llap/ext/TestLlapInputSplit.java       |  18 ++
 .../apache/hive/jdbc/TestJdbcWithMiniLlap.java  |  37 +----
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |   3 +-
 .../hadoop/hive/llap/LlapBaseRecordReader.java  |  38 +++--
 .../hadoop/hive/llap/LlapRowRecordReader.java   |  48 ++++--
 .../apache/hadoop/hive/llap/SubmitWorkInfo.java |  16 ++
 .../ext/LlapTaskUmbilicalExternalClient.java    |  46 +++---
 .../helpers/LlapTaskUmbilicalServer.java        |  16 ++
 .../hadoop/hive/llap/LlapBaseInputFormat.java   |  20 +--
 .../org/apache/hadoop/hive/llap/LlapDump.java   |  11 +-
 .../hadoop/hive/llap/LlapRowInputFormat.java    |  18 ++
 .../hive/llap/daemon/impl/LlapDaemon.java       |   2 +-
 .../llap/daemon/impl/TaskRunnerCallable.java    |   6 +-
 .../daemon/impl/TaskExecutorTestHelpers.java    |   2 +-
 .../hadoop/hive/llap/LlapDataOutputBuffer.java  | 165 -------------------
 .../hive/llap/LlapOutputFormatService.java      |  27 +--
 .../hive/ql/exec/tez/HiveSplitGenerator.java    |   4 +-
 .../hive/ql/io/HivePassThroughRecordWriter.java |   4 -
 .../hive/ql/parse/TypeCheckProcFactory.java     |   9 +-
 .../ql/udf/generic/GenericUDTFGetSplits.java    |   1 -
 .../org/apache/tez/dag/api/TaskSpecBuilder.java |  17 +-
 .../hadoop/hive/llap/TestLlapOutputFormat.java  |   2 +-
 .../results/clientpositive/show_functions.q.out |   1 +
 23 files changed, 209 insertions(+), 302 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
index 8264190..1de8aa6 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java
@@ -1,3 +1,21 @@
+/**
+ * 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.ext;
 
 import java.io.ByteArrayInputStream;

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
index 5b4ba49..48b9493 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java
@@ -161,39 +161,7 @@ public class TestJdbcWithMiniLlap {
     stmt.close();
   }
 
-  private static boolean timedOut = false;
-
-  private static class TestTimerTask extends TimerTask {
-    private boolean timedOut = false;
-    private Thread threadToInterrupt;
-
-    public TestTimerTask(Thread threadToInterrupt) {
-      this.threadToInterrupt = threadToInterrupt;
-    }
-
-    @Override
-    public void run() {
-      System.out.println("Test timed out!");
-      timedOut = true;
-      threadToInterrupt.interrupt();
-    }
-
-    public boolean isTimedOut() {
-      return timedOut;
-    }
-
-    public void setTimedOut(boolean timedOut) {
-      this.timedOut = timedOut;
-    }
-
-  }
-
   private int getLlapIFRowCount(String query, int numSplits) throws Exception {
-    // Add a timer task to stop this test if it has not finished in a reasonable amount of time.
-    Timer timer = new Timer();
-    long delay = 30000;
-    TestTimerTask timerTask = new TestTimerTask(Thread.currentThread());
-    timer.schedule(timerTask, delay);
 
     // Setup LlapInputFormat
     String url = miniHS2.getJdbcURL();
@@ -245,13 +213,10 @@ public class TestJdbcWithMiniLlap {
       }
     }
 
-    timer.cancel();
-    assertFalse("Test timed out", timerTask.isTimedOut());
-
     return rowCount;
   }
 
-  @Test
+  @Test(timeout = 60000)
   public void testLlapInputFormatEndToEnd() throws Exception {
     createTestTable("testtab1");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 3f8c3c5..a6e8efa 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -1468,8 +1468,7 @@ public class QTestUtil {
       ".*Input:.*/data/files/.*",
       ".*Output:.*/data/files/.*",
       ".*total number of created files now is.*",
-      ".*.hive-staging.*",
-      "table_.*"
+      ".*.hive-staging.*"
   });
 
   private final Pattern[] partialReservedPlanMask = toPattern(new String[] {

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
index 0cd9672..3c858a8 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
@@ -23,8 +23,10 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.DataInputStream;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.llap.Schema;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -37,22 +39,27 @@ import org.apache.hadoop.mapred.JobConf;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * Base LLAP RecordReader to handle receiving of the data from the LLAP daemon.
+ */
 public class LlapBaseRecordReader<V extends WritableComparable> implements RecordReader<NullWritable, V> {
   private static final Logger LOG = LoggerFactory.getLogger(LlapBaseRecordReader.class);
 
-  DataInputStream din;
-  Schema schema;
-  Class<V> clazz;
-
+  protected final DataInputStream din;
+  protected final Schema schema;
+  protected final Class<V> clazz;
 
   protected Thread readerThread = null;
-  protected LinkedBlockingQueue<ReaderEvent> readerEvents = new LinkedBlockingQueue<ReaderEvent>();
+  protected final LinkedBlockingQueue<ReaderEvent> readerEvents = new LinkedBlockingQueue<ReaderEvent>();
+  protected final long timeout;
 
-  public LlapBaseRecordReader(InputStream in, Schema schema, Class<V> clazz) {
+  public LlapBaseRecordReader(InputStream in, Schema schema, Class<V> clazz, JobConf job) {
     din = new DataInputStream(in);
     this.schema = schema;
     this.clazz = clazz;
     this.readerThread = Thread.currentThread();
+    this.timeout = 3 * HiveConf.getTimeVar(job,
+        HiveConf.ConfVars.LLAP_DAEMON_AM_LIVENESS_CONNECTION_TIMEOUT_MS, TimeUnit.MILLISECONDS);
   }
 
   public Schema getSchema() {
@@ -65,10 +72,16 @@ public class LlapBaseRecordReader<V extends WritableComparable> implements Recor
   }
 
   @Override
-  public long getPos() { return 0; }
+  public long getPos() {
+    // dummy impl
+    return 0;
+  }
 
   @Override
-  public float getProgress() { return 0f; }
+  public float getProgress() {
+    // dummy impl
+    return 0f;
+  }
 
   @Override
   public NullWritable createKey() {
@@ -106,7 +119,7 @@ public class LlapBaseRecordReader<V extends WritableComparable> implements Recor
     } catch (IOException io) {
       if (Thread.interrupted()) {
         // Either we were interrupted by one of:
-        // 1. handleEvent(), in which case there is a reader event waiting for us in the queue
+        // 1. handleEvent(), in which case there is a reader (error) event waiting for us in the queue
         // 2. Some other unrelated cause which interrupted us, in which case there may not be a reader event coming.
         // Either way we should not try to block trying to read the reader events queue.
         if (readerEvents.isEmpty()) {
@@ -186,9 +199,12 @@ public class LlapBaseRecordReader<V extends WritableComparable> implements Recor
     }
   }
 
-  protected ReaderEvent getReaderEvent() {
+  protected ReaderEvent getReaderEvent() throws IOException {
     try {
-      ReaderEvent event = readerEvents.take();
+      ReaderEvent event = readerEvents.poll(timeout, TimeUnit.MILLISECONDS);
+      if (event == null) {
+        throw new IOException("Timed out getting readerEvents");
+      }
       return event;
     } catch (InterruptedException ie) {
       throw new RuntimeException("Interrupted while getting readerEvents, not expected: " + ie.getMessage(), ie);

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
index 4e000ff..084da0a 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java
@@ -1,3 +1,21 @@
+/**
+ * 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 com.google.common.base.Preconditions;
@@ -32,21 +50,29 @@ import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
+/**
+ * Row-based record reader for LLAP.
+ */
 public class LlapRowRecordReader implements RecordReader<NullWritable, Row> {
 
   private static final Logger LOG = LoggerFactory.getLogger(LlapRowRecordReader.class);
 
-  Configuration conf;
-  RecordReader<NullWritable, Text> reader;
-  Schema schema;
-  SerDe serde;
-  final Text textData = new Text();
+  protected final Configuration conf;
+  protected final RecordReader<NullWritable, Text> reader;
+  protected final Schema schema;
+  protected final SerDe serde;
+  protected final Text textData = new Text();
 
-  public LlapRowRecordReader(Configuration conf, Schema schema, RecordReader<NullWritable, Text> reader) {
+  public LlapRowRecordReader(Configuration conf, Schema schema, RecordReader<NullWritable, Text> reader) throws IOException {
     this.conf = conf;
     this.schema = schema;
     this.reader = reader;
+
+    try {
+      serde = initSerDe(conf);
+    } catch (SerDeException err) {
+      throw new IOException(err);
+    }
   }
 
   @Override
@@ -78,14 +104,6 @@ public class LlapRowRecordReader implements RecordReader<NullWritable, Row> {
   public boolean next(NullWritable key, Row value) throws IOException {
     Preconditions.checkArgument(value != null);
 
-    if (serde == null) {
-      try {
-        serde = initSerDe(conf);
-      } catch (SerDeException err) {
-        throw new IOException(err);
-      }
-    }
-
     boolean hasNext = reader.next(key,  textData);
     if (hasNext) {
       // Deserialize Text to column values, and populate the row record

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java b/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
index 83149ab..6704294 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java
@@ -1,3 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.hadoop.hive.llap;
 
 import java.io.DataInput;

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/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 6e2c85d..0edb1cd 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,3 +1,19 @@
+/*
+ * 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.ext;
 
 import java.io.IOException;
@@ -83,10 +99,6 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
     }
   }
 
-  // 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());
@@ -96,9 +108,9 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
     this.sessionToken = sessionToken;
     this.responder = responder;
     this.timer = new ScheduledThreadPoolExecutor(1);
-    this.connectionTimeout = HiveConf.getTimeVar(conf,
+    this.connectionTimeout = 3 * 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.
+    // 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);
   }
@@ -173,24 +185,6 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
             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) {
@@ -261,7 +255,6 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
         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();
@@ -277,7 +270,6 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
         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?
       }
     }
   }
@@ -291,7 +283,7 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
 
 
 
-  // TODO Ideally, the server should be shared across all client sessions running on the same node.
+  // Ideally, the server should be shared across all client sessions running on the same node.
   private class LlapTaskUmbilicalExternalImpl implements  LlapTaskUmbilicalProtocol {
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java b/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
index dbd591a..79800da 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java
@@ -1,3 +1,19 @@
+/*
+ * 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.tezplugins.helpers;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/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 988002f..4306c22 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
@@ -97,6 +97,9 @@ import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
 
 
+/**
+ * Base LLAP input format to handle requesting of splits and communication with LLAP daemon.
+ */
 public class LlapBaseInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
 
   private static final Logger LOG = LoggerFactory.getLogger(LlapBaseInputFormat.class);
@@ -178,7 +181,7 @@ public class LlapBaseInputFormat<V extends WritableComparable> implements InputF
 
     LOG.info("Registered id: " + id);
 
-    LlapBaseRecordReader recordReader = new LlapBaseRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class);
+    LlapBaseRecordReader recordReader = new LlapBaseRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class, job);
     umbilicalResponder.setRecordReader(recordReader);
     return recordReader;
   }
@@ -312,21 +315,6 @@ public class LlapBaseInputFormat<V extends WritableComparable> implements InputF
 
     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);

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
----------------------------------------------------------------------
diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
index d485bfa..08ad1f5 100644
--- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
+++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
@@ -57,6 +57,11 @@ import org.apache.hadoop.hive.llap.LlapRowRecordReader;
 import org.apache.hadoop.hive.llap.Row;
 import org.apache.hadoop.hive.llap.Schema;
 
+/**
+ * Utility to test query and data retrieval via the LLAP input format.
+ * llapdump --hiveconf hive.zookeeper.quorum=localhost --hiveconf hive.zookeeper.client.port=2181 --hiveconf hive.llap.daemon.service.hosts=@llap_MiniLlapCluster 'select * from employee where employee_id < 10'
+ *
+ */
 public class LlapDump {
 
   private static final Logger LOG = LoggerFactory.getLogger(LlapDump.class);
@@ -64,7 +69,7 @@ public class LlapDump {
   private static String url = "jdbc:hive2://localhost:10000/default";
   private static String user = "hive";
   private static String pwd = "";
-  private static String query = "select * from test";
+  private static String query = null;
   private static String numSplits = "1";
 
   public static void main(String[] args) throws Exception {
@@ -99,6 +104,10 @@ public class LlapDump {
       query = cli.getArgs()[0];
     }
 
+    if (query == null) {
+      throw new IllegalArgumentException("No query string specified");
+    }
+
     System.out.println("url: "+url);
     System.out.println("user: "+user);
     System.out.println("query: "+query);

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
index 56ad555..7efc711 100644
--- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
+++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.hadoop.hive.llap;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/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 8b2b978..223c390 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
@@ -343,7 +343,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     if (webServices != null) {
       getConfig().setInt(ConfVars.LLAP_DAEMON_WEB_PORT.varname, webServices.getPort());
     }
-    LlapOutputFormatService.get();
+    getConfig().setInt(ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT.varname, LlapOutputFormatService.get().getPort());
 
     this.registry.init(getConfig());
     this.registry.start();

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/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 8594ee1..74359fa 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
@@ -141,8 +141,10 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
         vertex, request.getFragmentNumber(), request.getAttemptNumber(), attemptId);
     this.amReporter = amReporter;
     // Register with the AMReporter when the callable is setup. Unregister once it starts running.
-    this.amReporter.registerTask(request.getAmHost(), request.getAmPort(),
-        vertex.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
+    if (amReporter != null && jobToken != null) {
+      this.amReporter.registerTask(request.getAmHost(), request.getAmPort(),
+          vertex.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
+    }
     this.metrics = metrics;
     this.requestId = taskSpec.getTaskAttemptID().toString();
     // TODO Change this to the queryId/Name when that's available.

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/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 d699f20..279baf1 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
@@ -147,7 +147,7 @@ public class TaskExecutorTestHelpers {
     public MockRequest(SubmitWorkRequestProto requestProto, QueryFragmentInfo fragmentInfo,
                        boolean canFinish, long workTime) {
       super(requestProto, fragmentInfo, new Configuration(),
-          new ExecutionContextImpl("localhost"), null, new Credentials(), 0, null, null, mock(
+          new ExecutionContextImpl("localhost"), null, new Credentials(), 0, mock(AMReporter.class), null, mock(
               LlapDaemonExecutorMetrics.class),
           mock(KilledTaskHandler.class), mock(
               FragmentCompletionHandler.class), new DefaultHadoopShim(), null);

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/hadoop/hive/llap/LlapDataOutputBuffer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapDataOutputBuffer.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapDataOutputBuffer.java
deleted file mode 100644
index aad8968..0000000
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapDataOutputBuffer.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.ql.llap;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-/**
- * A thread-not-safe version of Hadoop's DataOutputBuffer, which removes all
- * synchronized modifiers.
- */
-public class LlapDataOutputBuffer implements DataOutput {
-
-  int readOffset;
-  int writeOffset;
-  byte[] buffer;
-
-  /** Constructs a new empty buffer. */
-  public LlapDataOutputBuffer(int length) {
-    buffer = new byte[length];
-    reset();
-  }
-
-  /**
-   * Returns the current contents of the buffer. Data is only valid to
-   * {@link #getLength()}.
-   */
-  public byte[] getData() {
-    return buffer;
-  }
-
-  /** Returns the length of the valid data currently in the buffer. */
-  public int getLength() {
-    return (writeOffset - readOffset) % buffer.length;
-  }
-
-  /** Resets the buffer to empty. */
-  public LlapDataOutputBuffer reset() {
-    readOffset = 0;
-    writeOffset = 0;
-    return this;
-  }
-
-  /** Writes bytes from a DataInput directly into the buffer. */
-  public void write(DataInput in, int length) throws IOException {
-    //
-  }
-
-  @Override
-  public synchronized void write(int b) throws IOException {
-    while (readOffset == writeOffset) {
-      try {
-	wait();
-      } catch(InterruptedException e) {
-      }
-    }
-    buffer[writeOffset] = (byte)b;
-    writeOffset = (writeOffset + 1) % buffer.length;
-    notify();
-  }
-
-  public synchronized int read() throws IOException {
-    while (readOffset == writeOffset) {
-      try {
-	wait();
-      } catch(InterruptedException e) {
-      }
-    }
-    int b = buffer[readOffset];
-    readOffset = (readOffset + 1) % buffer.length;
-    notify();
-    return b;
-  }
-
-  @Override
-  public void write(byte b[], int off, int len) throws IOException {
-    while(len-- != 0) {
-      write(b[off++]);
-    }
-  }
-
-  @Override
-  public void write(byte b[]) throws IOException {
-    write(b, 0, b.length);
-  }
-
-
-  @Override
-  public void writeBoolean(boolean v) throws IOException {
-    write(v?1:0);
-  }
-
-  @Override
-  public void writeByte(int v) throws IOException  {
-    write(v);
-  }
-
-  @Override
-  public void writeChar(int v) throws IOException  {
-    write(v);
-  }
-
-  @Override
-  public void writeBytes(String v) throws IOException  {
-    write(v.getBytes(), 0, v.length());
-  }
-
-  @Override
-  public void writeChars(String v) throws IOException  {
-    write(v.getBytes(), 0, v.length());
-  }
-
-  @Override
-  public void writeDouble(double v) throws IOException  {
-    write(ByteBuffer.allocate(8).putDouble(v).array(),0,8);
-  }
-
-  @Override
-  public void writeFloat(float v) throws IOException  {
-    write(ByteBuffer.allocate(4).putFloat(v).array(),0,4);
-  }
-
-  @Override
-  public void writeInt(int v) throws IOException  {
-    write(v);
-    write(v>>>8);
-    write(v>>>16);
-    write(v>>>24);
-  }
-
-  @Override
-  public void writeLong(long v) throws IOException  {
-    int v1 = (int)v;
-    int v2 = (int)v>>>32;
-    write(v1);
-    write(v2);
-  }
-
-  @Override
-  public void writeShort(int v) throws IOException  {
-    write(v);
-    write(v>>>8);
-  }
-
-  @Override
-  public void writeUTF(String v) throws IOException  {
-    write(v.getBytes(), 0, v.length());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
index b39f085..6adbf7c 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
@@ -20,6 +20,7 @@ import java.util.Map;
 import java.util.HashMap;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.net.InetSocketAddress;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -60,7 +61,7 @@ import io.netty.util.concurrent.Future;
 
 
 /**
- *
+ * Responsible for sending back result set data to the connections made by external clients via the LLAP input format.
  */
 public class LlapOutputFormatService {
 
@@ -75,6 +76,7 @@ public class LlapOutputFormatService {
   private EventLoopGroup eventLoopGroup;
   private ServerBootstrap serverBootstrap;
   private ChannelFuture listeningChannelFuture;
+  private int port;
 
   private LlapOutputFormatService() throws IOException {
     writers = new HashMap<String, RecordWriter>();
@@ -92,17 +94,18 @@ public class LlapOutputFormatService {
   public void start() throws IOException {
     LOG.info("Starting LlapOutputFormatService");
 
-    int port = conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
+    int portFromConf = conf.getIntVar(HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT);
     eventLoopGroup = new NioEventLoopGroup(1);
     serverBootstrap = new ServerBootstrap();
     serverBootstrap.group(eventLoopGroup);
     serverBootstrap.channel(NioServerSocketChannel.class);
     serverBootstrap.childHandler(new LlapOutputFormatServiceChannelHandler());
     try {
-      LOG.info("LlapOutputFormatService: Binding to port " + port);
-      listeningChannelFuture = serverBootstrap.bind(port).sync();
+      listeningChannelFuture = serverBootstrap.bind(portFromConf).sync();
+      this.port = ((InetSocketAddress) listeningChannelFuture.channel().localAddress()).getPort();
+      LOG.info("LlapOutputFormatService: Binding to port " + this.port);
     } catch (InterruptedException err) {
-      throw new IOException("LlapOutputFormatService: Error binding to port " + port, err);
+      throw new IOException("LlapOutputFormatService: Error binding to port " + portFromConf, err);
     }
   }
 
@@ -132,6 +135,10 @@ public class LlapOutputFormatService {
     return writer;
   }
 
+  public int getPort() {
+    return port;
+  }
+
   protected class LlapOutputFormatServiceHandler extends SimpleChannelInboundHandler<String> {
     @Override
     public void channelRead0(ChannelHandlerContext ctx, String msg) {
@@ -179,11 +186,11 @@ public class LlapOutputFormatService {
   protected class LlapOutputFormatServiceChannelHandler extends ChannelInitializer<SocketChannel> {
     @Override
     public void initChannel(SocketChannel ch) throws Exception {
-        ch.pipeline().addLast(
-            new DelimiterBasedFrameDecoder(MAX_QUERY_ID_LENGTH, Delimiters.nulDelimiter()),
-            new StringDecoder(),
-            new StringEncoder(),
-            new LlapOutputFormatServiceHandler());
+      ch.pipeline().addLast(
+          new DelimiterBasedFrameDecoder(MAX_QUERY_ID_LENGTH, Delimiters.nulDelimiter()),
+          new StringDecoder(),
+          new StringEncoder(),
+          new LlapOutputFormatServiceHandler());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
index 4e6272f..54fc3af 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
@@ -93,7 +93,7 @@ public class HiveSplitGenerator extends InputInitializer {
     this.work = work;
     this.jobConf = new JobConf(conf);
 
-    // TODO RSHACK - assuming grouping enabled always.
+    // Assuming grouping enabled always.
     userPayloadProto = MRInputUserPayloadProto.newBuilder().setGroupingEnabled(true).build();
 
     this.splitLocationProvider = Utils.getSplitLocationProvider(conf, LOG);
@@ -106,7 +106,7 @@ public class HiveSplitGenerator extends InputInitializer {
     // must be setup and initialized here so that it sets up it's structures to start accepting events.
     // Setting it up in initialize leads to a window where events may come in before the pruner is
     // initialized, which may cause it to drop events.
-    // TODO RSHACK - No dynamic partition pruning
+    // No dynamic partition pruning
     pruner = null;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java
index 6d00a0a..454c321 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HivePassThroughRecordWriter.java
@@ -23,14 +23,11 @@ import java.io.IOException;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 public class HivePassThroughRecordWriter <K extends WritableComparable<?>, V extends Writable>
 implements RecordWriter {
 
-  public static final Logger LOG = LoggerFactory.getLogger(HivePassThroughRecordWriter.class);
   private final org.apache.hadoop.mapred.RecordWriter<K, V> mWriter;
 
   public HivePassThroughRecordWriter(org.apache.hadoop.mapred.RecordWriter<K, V> writer) {
@@ -45,7 +42,6 @@ implements RecordWriter {
 
   @Override
   public void close(boolean abort) throws IOException {
-    LOG.info("Closing the pass through writer.");
     //close with null reporter
     mWriter.close(null);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
index 81320a2..19e80f7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
@@ -1277,17 +1277,14 @@ public class TypeCheckProcFactory {
       try {
         return getXpathOrFuncExprNodeDesc(expr, isFunction, children, ctx);
       } catch (UDFArgumentTypeException e) {
-        LOG.error("UDFArgumentTypeException: ", e);
         throw new SemanticException(ErrorMsg.INVALID_ARGUMENT_TYPE.getMsg(expr
-            .getChild(childrenBegin + e.getArgumentId()), e.getMessage()));
+            .getChild(childrenBegin + e.getArgumentId()), e.getMessage()), e);
       } catch (UDFArgumentLengthException e) {
-        LOG.error("UDFArgumentLengthException: ", e);
         throw new SemanticException(ErrorMsg.INVALID_ARGUMENT_LENGTH.getMsg(
-            expr, e.getMessage()));
+            expr, e.getMessage()), e);
       } catch (UDFArgumentException e) {
-        LOG.error("UDFArgumentException: ", e);
         throw new SemanticException(ErrorMsg.INVALID_ARGUMENT.getMsg(expr, e
-            .getMessage()));
+            .getMessage()), e);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
index 50cdadb..ce69ee6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
@@ -277,7 +277,6 @@ public class GenericUDTFGetSplits extends GenericUDTF {
 
     DAG dag = DAG.create(work.getName());
     dag.setCredentials(job.getCredentials());
-    // TODO: set access control? TezTask.setAccessControlsForCurrentUser(dag);
 
     DagUtils utils = DagUtils.getInstance();
     Context ctx = new Context(job);

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java b/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
index 5db8c48..6d31802 100644
--- a/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
+++ b/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java
@@ -1,3 +1,19 @@
+/*
+ * 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.tez.dag.api;
 
 import java.util.ArrayList;
@@ -25,7 +41,6 @@ public class TaskSpecBuilder {
     List<RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>> outputs =
         vertex.getOutputs();
 
-    // TODO RSHACK - for now these must be of size 1.
     Preconditions.checkState(inputs.size() == 1);
     Preconditions.checkState(outputs.size() == 1);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
index 37e21b8..907d5b0 100644
--- a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java
@@ -113,7 +113,7 @@ public class TestLlapOutputFormat {
       writer.close(null);
 
       InputStream in = socket.getInputStream();
-      LlapBaseRecordReader reader = new LlapBaseRecordReader(in, null, Text.class);
+      LlapBaseRecordReader reader = new LlapBaseRecordReader(in, null, Text.class, job);
 
       LOG.debug("Have record reader");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2a03f1f4/ql/src/test/results/clientpositive/show_functions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_functions.q.out b/ql/src/test/results/clientpositive/show_functions.q.out
index 5c8b982..a811747 100644
--- a/ql/src/test/results/clientpositive/show_functions.q.out
+++ b/ql/src/test/results/clientpositive/show_functions.q.out
@@ -91,6 +91,7 @@ format_number
 from_unixtime
 from_utc_timestamp
 get_json_object
+get_splits
 greatest
 hash
 hex


[26/39] hive git commit: Merge remote-tracking branch 'origin/master' into llap

Posted by jd...@apache.org.
Merge remote-tracking branch 'origin/master' into llap


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/390cb8cd
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/390cb8cd
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/390cb8cd

Branch: refs/heads/master
Commit: 390cb8cd0259c1830f5d42c3ca63829bcd7022ec
Parents: ffed821 0ebcd93
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu Apr 28 15:09:59 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu Apr 28 15:09:59 2016 -0700

----------------------------------------------------------------------
 bin/ext/hiveserver.cmd                          |   48 -
 bin/ext/hiveserver.sh                           |   35 -
 .../common/metrics/common/MetricsConstant.java  |    2 +
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   23 +-
 data/conf/hive-site.xml                         |    2 +-
 .../hive/hcatalog/streaming/HiveEndPoint.java   |    3 +-
 .../hive/hcatalog/streaming/TestStreaming.java  |    2 +
 .../TestHiveAuthorizerCheckInvocation.java      |   52 +
 .../hive/llap/registry/ServiceInstanceSet.java  |    6 +
 .../registry/impl/LlapFixedRegistryImpl.java    |    5 +
 .../impl/LlapZookeeperRegistryImpl.java         |    5 +
 .../hadoop/hive/llap/cache/BuddyAllocator.java  |    5 +-
 .../llap/cache/LowLevelLrfuCachePolicy.java     |    9 +-
 .../llap/daemon/impl/ContainerRunnerImpl.java   |    7 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |   20 +-
 .../llap/daemon/impl/TaskExecutorService.java   |   53 +-
 .../llap/daemon/impl/TaskRunnerCallable.java    |   28 +-
 .../hive/llap/io/api/impl/LlapIoImpl.java       |    1 -
 .../llap/io/decode/EncodedDataConsumer.java     |    5 -
 .../llap/io/encoded/OrcEncodedDataReader.java   |    3 -
 .../hive/llap/io/metadata/OrcMetadataCache.java |   42 +-
 .../hive/llap/metrics/LlapDaemonCacheInfo.java  |    6 +-
 .../llap/metrics/LlapDaemonCacheMetrics.java    |   35 +-
 .../llap/metrics/LlapDaemonExecutorInfo.java    |   23 +-
 .../llap/metrics/LlapDaemonExecutorMetrics.java |  152 +-
 .../hive/llap/metrics/LlapDaemonIOInfo.java     |    4 -
 .../hive/llap/metrics/LlapDaemonIOMetrics.java  |   37 +-
 .../hive/llap/cache/TestOrcMetadataCache.java   |   17 +-
 .../daemon/impl/TestTaskExecutorService.java    |    2 +-
 .../tezplugins/LlapTaskSchedulerService.java    |  234 +-
 .../metrics/LlapTaskSchedulerInfo.java          |   59 +
 .../metrics/LlapTaskSchedulerMetrics.java       |  197 +
 .../TestLlapTaskSchedulerService.java           |    2 +-
 metastore/pom.xml                               |    8 +
 .../hadoop/hive/metastore/ObjectStore.java      |   19 +-
 .../metastore/txn/CompactionTxnHandler.java     |    3 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |    2 +
 .../hadoop/hive/metastore/TestObjectStore.java  |   50 +
 pom.xml                                         |    2 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |    2 +-
 .../hadoop/hive/ql/exec/FetchOperator.java      |   16 +-
 .../apache/hadoop/hive/ql/exec/FetchTask.java   |    4 +
 .../ql/optimizer/ColumnPrunerProcFactory.java   |    2 +-
 .../hive/ql/optimizer/SimpleFetchOptimizer.java |    2 +-
 .../calcite/rules/HiveRelFieldTrimmer.java      |    2 +-
 .../hive/ql/parse/ColumnAccessAnalyzer.java     |   31 +-
 .../hive/ql/parse/ImportSemanticAnalyzer.java   |   25 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   12 +-
 .../hadoop/hive/ql/parse/TestColumnAccess.java  |   10 +-
 .../clientpositive/lateral_view_noalias.q.out   |  120 +-
 .../results/clientpositive/nonmr_fetch.q.out    |   78 +-
 .../clientpositive/select_dummy_source.q.out    |   38 +-
 .../clientpositive/tez/explainuser_1.q.out      |   14 +-
 .../clientpositive/tez/explainuser_3.q.out      |   38 +-
 .../tez/select_dummy_source.q.out               |   76 +-
 .../results/clientpositive/udf_explode.q.out    |  208 +-
 .../results/clientpositive/udf_inline.q.out     |   42 +-
 .../results/clientpositive/udtf_explode.q.out   |  150 +-
 service/if/hive_service.thrift                  |   86 -
 service/src/gen/thrift/gen-cpp/ThriftHive.cpp   | 3544 --------
 service/src/gen/thrift/gen-cpp/ThriftHive.h     | 1224 ---
 .../gen-cpp/ThriftHive_server.skeleton.cpp      |   84 -
 .../thrift/gen-cpp/hive_service_constants.cpp   |   17 -
 .../gen/thrift/gen-cpp/hive_service_constants.h |   24 -
 .../gen/thrift/gen-cpp/hive_service_types.cpp   |  351 -
 .../src/gen/thrift/gen-cpp/hive_service_types.h |  176 -
 .../hadoop/hive/service/HiveClusterStatus.java  |  901 --
 .../hive/service/HiveServerException.java       |  601 --
 .../hadoop/hive/service/JobTrackerState.java    |   45 -
 .../apache/hadoop/hive/service/ThriftHive.java  | 7784 ------------------
 service/src/gen/thrift/gen-php/ThriftHive.php   | 1943 -----
 service/src/gen/thrift/gen-php/Types.php        |  338 -
 .../gen-py/hive_service/ThriftHive-remote       | 1263 ---
 .../thrift/gen-py/hive_service/ThriftHive.py    | 1674 ----
 .../gen/thrift/gen-py/hive_service/__init__.py  |    1 -
 .../gen/thrift/gen-py/hive_service/constants.py |   11 -
 .../gen/thrift/gen-py/hive_service/ttypes.py    |  260 -
 .../gen/thrift/gen-rb/hive_service_constants.rb |    9 -
 .../src/gen/thrift/gen-rb/hive_service_types.rb |   68 -
 service/src/gen/thrift/gen-rb/thrift_hive.rb    |  555 --
 .../TestScripts.testPrepGit.approved.txt        |    2 -
 .../TestScripts.testPrepHadoop1.approved.txt    |    2 -
 .../TestScripts.testPrepNone.approved.txt       |    2 -
 .../TestScripts.testPrepSvn.approved.txt        |    2 -
 84 files changed, 1191 insertions(+), 21854 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapFixedRegistryImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/390cb8cd/pom.xml
----------------------------------------------------------------------


[24/39] hive git commit: HIVE-13594: Misc cleanup on llap branch

Posted by jd...@apache.org.
HIVE-13594: Misc cleanup on llap branch


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/53b43cd4
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/53b43cd4
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/53b43cd4

Branch: refs/heads/master
Commit: 53b43cd440a2ee32efa5ad62f2684a4578f390ca
Parents: e69bd1e
Author: Jason Dere <jd...@hortonworks.com>
Authored: Fri Apr 22 14:33:36 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Fri Apr 22 14:33:36 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  3 --
 .../test/resources/testconfiguration.properties |  3 +-
 jdbc/pom.xml                                    | 16 +++----
 .../hadoop/hive/llap/LlapBaseRecordReader.java  |  2 +-
 .../ext/LlapTaskUmbilicalExternalClient.java    |  4 +-
 .../org/apache/hadoop/hive/llap/LlapDump.java   |  3 +-
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |  3 --
 .../hive/ql/exec/SerializationUtilities.java    |  2 +-
 .../hive/ql/exec/tez/HiveSplitGenerator.java    | 23 ++++-----
 .../hive/ql/optimizer/SimpleFetchOptimizer.java | 50 +++++++++++---------
 .../ql/udf/generic/GenericUDTFGetSplits.java    |  6 +--
 11 files changed, 51 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/53b43cd4/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index fa724ae..c8c26db 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2763,9 +2763,6 @@ public class HiveConf extends Configuration {
     LLAP_DAEMON_OUTPUT_SERVICE_PORT("hive.llap.daemon.output.service.port", 15003,
         "LLAP daemon output service port"),
 
-    LLAP_TMP_SUBMITWORK_USING_TEZ_AM("hive.llap.tmp.submit.work.using.tez.am", true,""),
-    LLAP_TMP_EXT_CLIENT_NUM_SERVER_HANDLERS("hive.llap.tmp.ext.client.num.server.handlers", 1, ""),
-
     SPARK_CLIENT_FUTURE_TIMEOUT("hive.spark.client.future.timeout",
       "60s", new TimeValidator(TimeUnit.SECONDS),
       "Timeout for requests from Hive client to remote Spark driver."),

http://git-wip-us.apache.org/repos/asf/hive/blob/53b43cd4/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 1669b9c..e46e6ce 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -503,8 +503,7 @@ minillap.shared.query.files=bucket_map_join_tez1.q,\
   vectorized_dynamic_partition_pruning.q,\
   tez_multi_union.q,\
   tez_join.q,\
-  tez_union_multiinsert.q,\
-  udtf_get_splits.q
+  tez_union_multiinsert.q
 
 
 minillap.query.files=llap_udf.q

http://git-wip-us.apache.org/repos/asf/hive/blob/53b43cd4/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index c99a351..f87ab59 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -42,13 +42,14 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
-      <artifactId>hive-exec</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hive</groupId>
       <artifactId>hive-service</artifactId>
       <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hive</groupId>
+            <artifactId>hive-exec</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
@@ -70,11 +71,6 @@
       <artifactId>hive-service-rpc</artifactId>
       <version>${project.version}</version>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hive</groupId>
-      <artifactId>hive-llap-common</artifactId>
-      <version>${project.version}</version>
-    </dependency>
     <!-- inter-project -->
     <dependency>
       <groupId>org.apache.httpcomponents</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/53b43cd4/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
index 7073280..0cd9672 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java
@@ -191,7 +191,7 @@ public class LlapBaseRecordReader<V extends WritableComparable> implements Recor
       ReaderEvent event = readerEvents.take();
       return event;
     } catch (InterruptedException ie) {
-      throw new RuntimeException("Interrupted while getting readerEvents, not expected", ie);
+      throw new RuntimeException("Interrupted while getting readerEvents, not expected: " + ie.getMessage(), ie);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/53b43cd4/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 7d06637..8598bc8 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
@@ -102,8 +102,8 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
 
   @Override
   public void serviceStart() throws IOException {
-    int numHandlers = HiveConf.getIntVar(conf,
-        HiveConf.ConfVars.LLAP_TMP_EXT_CLIENT_NUM_SERVER_HANDLERS);
+    // 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();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/53b43cd4/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
----------------------------------------------------------------------
diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
index 1c4397f..d485bfa 100644
--- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
+++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java
@@ -40,7 +40,6 @@ import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.ql.io.RCFile.Reader;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
@@ -74,7 +73,7 @@ public class LlapDump {
 
     if (cli.hasOption('h')) {
       HelpFormatter formatter = new HelpFormatter();
-      formatter.printHelp("orcfiledump", opts);
+      formatter.printHelp("llapdump", opts);
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/53b43cd4/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index d2dfbb7..ec6381b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@ -36,13 +36,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.llap.LlapOutputFormat;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
-import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
@@ -203,7 +201,6 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
           }
         }
       }
-
       try {
         for (int i = 0; i < updaters.length; i++) {
           if (updaters[i] != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/53b43cd4/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
index eaa4293..b05a79e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
@@ -570,7 +570,7 @@ public class SerializationUtilities {
    * @param plan Usually of type MapredWork, MapredLocalWork etc.
    * @param out stream in which serialized plan is written into
    */
-  public static void serializeObjectByKryo(Kryo kryo, Object plan, OutputStream out) {
+  private static void serializeObjectByKryo(Kryo kryo, Object plan, OutputStream out) {
     Output output = new Output(out);
     kryo.setClassLoader(Utilities.getSessionSpecifiedClassLoader());
     kryo.writeObject(output, plan);

http://git-wip-us.apache.org/repos/asf/hive/blob/53b43cd4/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
index b16368f..4e6272f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
@@ -78,17 +78,16 @@ public class HiveSplitGenerator extends InputInitializer {
 
   private static final Logger LOG = LoggerFactory.getLogger(HiveSplitGenerator.class);
 
-  private DynamicPartitionPruner pruner = null;
-  private Configuration conf = null;
-  private JobConf jobConf = null;
-  private MRInputUserPayloadProto userPayloadProto = null;
-  private MapWork work = null;
+  private final DynamicPartitionPruner pruner;
+  private final Configuration conf;
+  private final JobConf jobConf;
+  private final MRInputUserPayloadProto userPayloadProto;
+  private final MapWork work;
   private final SplitGrouper splitGrouper = new SplitGrouper();
-  private SplitLocationProvider splitLocationProvider = null;
+  private final SplitLocationProvider splitLocationProvider;
 
-  
-  // TODO RSHACK This entire method needs to be reworked. Put back final fields, separate into reusable components etc.
-  public void initializeSplitGenerator(Configuration conf, MapWork work) throws IOException {
+  public HiveSplitGenerator(Configuration conf, MapWork work) throws IOException {
+    super(null);
 
     this.conf = conf;
     this.work = work;
@@ -103,8 +102,6 @@ public class HiveSplitGenerator extends InputInitializer {
     // Read all credentials into the credentials instance stored in JobConf.
     ShimLoader.getHadoopShims().getMergedCredentials(jobConf);
 
-    this.work = Utilities.getMapWork(jobConf);
-
     // Events can start coming in the moment the InputInitializer is created. The pruner
     // must be setup and initialized here so that it sets up it's structures to start accepting events.
     // Setting it up in initialize leads to a window where events may come in before the pruner is
@@ -116,9 +113,7 @@ public class HiveSplitGenerator extends InputInitializer {
   public HiveSplitGenerator(InputInitializerContext initializerContext) throws IOException,
       SerDeException {
     super(initializerContext);
-    if (initializerContext == null) {
-      return;
-    }
+
     Preconditions.checkNotNull(initializerContext);
     userPayloadProto =
         MRInputHelpers.parseMRInputPayload(initializerContext.getInputUserPayload());

http://git-wip-us.apache.org/repos/asf/hive/blob/53b43cd4/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
index ca8dccf..b5ceb14 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.hive.ql.exec.FetchTask;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.LimitOperator;
-import org.apache.hadoop.hive.ql.exec.UDTFOperator;
 import org.apache.hadoop.hive.ql.exec.ListSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorFactory;
@@ -141,31 +140,35 @@ public class SimpleFetchOptimizer extends Transform {
   }
 
   private boolean checkThreshold(FetchData data, int limit, ParseContext pctx) throws Exception {
-    boolean result = false;
-
     if (limit > 0) {
       if (data.hasOnlyPruningFilter()) {
         /* partitioned table + query has only pruning filters */
-        result = true;
+        return true;
       } else if (data.isPartitioned() == false && data.isFiltered() == false) {
         /* unpartitioned table + no filters */
-        result = true;
+        return true;
       }
       /* fall through */
-    } else {
-      long threshold = HiveConf.getLongVar(pctx.getConf(),
-	  HiveConf.ConfVars.HIVEFETCHTASKCONVERSIONTHRESHOLD);
-      if (threshold < 0) {
-	result = true;
-      } else {
-	long remaining = threshold;
-	remaining -= data.getInputLength(pctx, remaining);
-	if (remaining >= 0) {
-	  result = true;
-	}
+    }
+    long threshold = HiveConf.getLongVar(pctx.getConf(),
+        HiveConf.ConfVars.HIVEFETCHTASKCONVERSIONTHRESHOLD);
+    if (threshold < 0) {
+      return true;
+    }
+    Operator child = data.scanOp.getChildOperators().get(0);
+    if(child instanceof SelectOperator) {
+      // select *, constant and casts can be allowed without a threshold check
+      if (checkExpressions((SelectOperator)child)) {
+        return true;
       }
     }
-    return result;
+    long remaining = threshold;
+    remaining -= data.getInputLength(pctx, remaining);
+    if (remaining < 0) {
+      LOG.info("Threshold " + remaining + " exceeded for pseudoMR mode");
+      return false;
+    }
+    return true;
   }
 
   // all we can handle is LimitOperator, FilterOperator SelectOperator and final FS
@@ -184,20 +187,23 @@ public class SimpleFetchOptimizer extends Transform {
       return null;
     }
     Table table = ts.getConf().getTableMetadata();
+    if (table == null) {
+      return null;
+    }
     ReadEntity parent = PlanUtils.getParentViewInfo(alias, pctx.getViewAliasToInput());
-    if (table != null && !table.isPartitioned()) {
+    if (!table.isPartitioned()) {
       FetchData fetch = new FetchData(ts, parent, table, splitSample);
       return checkOperators(fetch, aggressive, false);
     }
 
     boolean bypassFilter = false;
-    if (table != null && HiveConf.getBoolVar(pctx.getConf(), HiveConf.ConfVars.HIVEOPTPPD)) {
+    if (HiveConf.getBoolVar(pctx.getConf(), HiveConf.ConfVars.HIVEOPTPPD)) {
       ExprNodeDesc pruner = pctx.getOpToPartPruner().get(ts);
       if (PartitionPruner.onlyContainsPartnCols(table, pruner)) {
         bypassFilter = !pctx.getPrunedPartitions(alias, ts).hasUnknownPartitions();
       }
     }
-    if (table != null && !aggressive && !bypassFilter) {
+    if (!aggressive && !bypassFilter) {
       return null;
     }
     PrunedPartitionList partitions = pctx.getPrunedPartitions(alias, ts);
@@ -225,7 +231,7 @@ public class SimpleFetchOptimizer extends Transform {
         continue;
       }
 
-      if (!(op instanceof LimitOperator || (op instanceof FilterOperator && bypassFilter)) || op instanceof UDTFOperator) {
+      if (!(op instanceof LimitOperator || (op instanceof FilterOperator && bypassFilter))) {
         break;
       }
 
@@ -283,7 +289,7 @@ public class SimpleFetchOptimizer extends Transform {
 
   private boolean isConvertible(FetchData fetch, Operator<?> operator, Set<Operator<?>> traversed) {
     if (operator instanceof ReduceSinkOperator || operator instanceof CommonJoinOperator
-        || operator instanceof ScriptOperator) {
+        || operator instanceof ScriptOperator || operator instanceof UDTFOperator) {
       return false;
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/53b43cd4/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
index 9a52c7d..2d36e5c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
@@ -298,15 +298,13 @@ public class GenericUDTFGetSplits extends GenericUDTF {
 
 
       // we have the dag now proceed to get the splits:
-      HiveSplitGenerator splitGenerator = new HiveSplitGenerator(null);
       Preconditions.checkState(HiveConf.getBoolVar(wxConf,
               HiveConf.ConfVars.HIVE_TEZ_GENERATE_CONSISTENT_SPLITS));
       Preconditions.checkState(HiveConf.getBoolVar(wxConf,
               HiveConf.ConfVars.LLAP_CLIENT_CONSISTENT_SPLITS));
-      splitGenerator.initializeSplitGenerator(wxConf, mapWork);
+      HiveSplitGenerator splitGenerator = new HiveSplitGenerator(wxConf, mapWork);
       List<Event> eventList = splitGenerator.initialize();
 
-      // hack - just serializing with kryo for now. This needs to be done properly
       InputSplit[] result = new InputSplit[eventList.size() - 1];
       DataOutputBuffer dob = new DataOutputBuffer();
 
@@ -458,7 +456,7 @@ public class GenericUDTFGetSplits extends GenericUDTF {
         break;
       case VARCHAR:
         VarcharTypeInfo varcharTypeInfo = (VarcharTypeInfo) typeInfo;
-        typeDesc = new TypeDesc(TypeDesc.Type.CHAR, varcharTypeInfo.getLength());
+        typeDesc = new TypeDesc(TypeDesc.Type.VARCHAR, varcharTypeInfo.getLength());
         break;
       case DATE:
         typeDesc = new TypeDesc(TypeDesc.Type.DATE);


[13/39] hive git commit: HIVE-13389: LLAP external submission client ends up attempting to find an LLAP instance based on the submitting user instead of the hive user

Posted by jd...@apache.org.
HIVE-13389: LLAP external submission client ends up attempting to find an LLAP instance based on the submitting user instead of the hive user


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/28d1082b
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/28d1082b
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/28d1082b

Branch: refs/heads/master
Commit: 28d1082b4a2acfa46beddc52e4d3944e783b6f21
Parents: 134f2f7
Author: Jason Dere <jd...@hortonworks.com>
Authored: Wed Mar 30 14:35:23 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Wed Mar 30 14:35:23 2016 -0700

----------------------------------------------------------------------
 .../src/java/org/apache/hadoop/hive/conf/HiveConf.java   |  3 +++
 .../org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java  |  4 +++-
 .../llap/registry/impl/LlapZookeeperRegistryImpl.java    |  9 ++++++++-
 .../org/apache/hadoop/hive/llap/LlapInputFormat.java     |  3 +++
 ql/pom.xml                                               |  6 ++++++
 .../java/org/apache/hadoop/hive/llap/LlapInputSplit.java | 11 +++++++++--
 .../hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java |  4 +++-
 7 files changed, 35 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/28d1082b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 88b4387..ef32b2b 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2591,6 +2591,9 @@ public class HiveConf extends Configuration {
         "ZooKeeper for ZooKeeper SecretManager."),
     LLAP_ZKSM_ZK_CONNECTION_STRING("hive.llap.zk.sm.connectionString", "",
         "ZooKeeper connection string for ZooKeeper SecretManager."),
+    LLAP_ZK_REGISTRY_USER("hive.llap.zk.registry.user", "",
+        "In the LLAP ZooKeeper-based registry, specifies the username in the Zookeeper path.\n" +
+        "This should be the hive user or whichever user is running the LLAP daemon."),
     // Note: do not rename to ..service.acl; Hadoop generates .hosts setting name from this,
     // resulting in a collision with existing hive.llap.daemon.service.hosts and bizarre errors.
     LLAP_SECURITY_ACL("hive.llap.daemon.acl", "*", "The ACL for LLAP daemon."),

http://git-wip-us.apache.org/repos/asf/hive/blob/28d1082b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
index b8b69a7..338930e 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestLlapInputSplit.java
@@ -46,7 +46,8 @@ public class TestLlapInputSplit {
         planBytes,
         fragmentBytes,
         locations,
-        schema);
+        schema,
+        "hive");
     ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream();
     DataOutputStream dataOut = new DataOutputStream(byteOutStream);
     split1.write(dataOut);
@@ -94,6 +95,7 @@ public class TestLlapInputSplit {
     }
     assertArrayEquals(split1.getLocations(), split2.getLocations());
     assertEquals(split1.getSchema(), split2.getSchema());
+    assertEquals(split1.getLlapUser(), split2.getLlapUser());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/28d1082b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
index c440e1e..0536589 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
@@ -176,7 +176,7 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
     // worker does not respond due to communication interruptions it will retain the same sequence
     // number when it returns back. If session timeout expires, the node will be deleted and new
     // addition of the same node (restart) will get next sequence number
-    this.pathPrefix = "/" + RegistryUtils.currentUser() + "/" + instanceName + "/workers/worker-";
+    this.pathPrefix = "/" + getZkPathUser(this.conf) + "/" + instanceName + "/workers/worker-";
     this.instancesCache = null;
     this.instances = null;
     this.stateChangeListeners = new HashSet<>();
@@ -210,6 +210,13 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
     return quorum.toString();
   }
 
+  private String getZkPathUser(Configuration conf) {
+    // External LLAP clients would need to set LLAP_ZK_REGISTRY_USER to the LLAP daemon user (hive),
+    // rather than relying on RegistryUtils.currentUser().
+    String user = HiveConf.getVar(conf, ConfVars.LLAP_ZK_REGISTRY_USER, RegistryUtils.currentUser());
+    return user;
+  }
+
   public Endpoint getRpcEndpoint() {
     final int rpcPort = HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_RPC_PORT);
     return RegistryTypeUtils.ipcEndpoint(IPC_LLAP, new InetSocketAddress(hostname, rpcPort));

http://git-wip-us.apache.org/repos/asf/hive/blob/28d1082b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
index 7f11e1d..aaca7d6 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
@@ -88,6 +88,9 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
       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);

http://git-wip-us.apache.org/repos/asf/hive/blob/28d1082b/ql/pom.xml
----------------------------------------------------------------------
diff --git a/ql/pom.xml b/ql/pom.xml
index ebb9599..e0de1fb 100644
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@ -286,6 +286,12 @@
       <version>${hadoop.version}</version>
       <optional>true</optional>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-registry</artifactId>
+      <version>${hadoop.version}</version>
+      <optional>true</optional>
+    </dependency>
 
     <dependency>
       <groupId>org.apache.ivy</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/28d1082b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
index 2ac0ccd..17a0d2d 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java
@@ -33,16 +33,18 @@ public class LlapInputSplit implements InputSplitWithLocationInfo {
   byte[] fragmentBytes;
   SplitLocationInfo[] locations;
   Schema schema;
+  String llapUser;
 
   public LlapInputSplit() {
   }
 
-  public LlapInputSplit(int splitNum, byte[] planBytes, byte[] fragmentBytes, SplitLocationInfo[] locations, Schema schema) {
+  public LlapInputSplit(int splitNum, byte[] planBytes, byte[] fragmentBytes, SplitLocationInfo[] locations, Schema schema, String llapUser) {
     this.planBytes = planBytes;
     this.fragmentBytes = fragmentBytes;
     this.locations = locations;
     this.schema = schema;
     this.splitNum = splitNum;
+    this.llapUser = llapUser;
   }
 
   public Schema getSchema() {
@@ -102,7 +104,7 @@ public class LlapInputSplit implements InputSplitWithLocationInfo {
       throw new IOException(e);
     }
 
-
+    out.writeUTF(llapUser);
   }
 
   @Override
@@ -134,10 +136,15 @@ public class LlapInputSplit implements InputSplitWithLocationInfo {
     } catch (Exception e) {
       throw new IOException(e);
     }
+    llapUser = in.readUTF();
   }
 
   @Override
   public SplitLocationInfo[] getLocationInfo() throws IOException {
     return locations;
   }
+
+  public String getLlapUser() {
+    return llapUser;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/28d1082b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
index ebb0ca5..6267324 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
@@ -75,6 +75,7 @@ import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SplitLocationInfo;
+import org.apache.hadoop.registry.client.binding.RegistryUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
@@ -319,6 +320,7 @@ public class GenericUDTFGetSplits extends GenericUDTF {
       ApplicationId fakeApplicationId
         = ApplicationId.newInstance(Math.abs(new Random().nextInt()), 0);
 
+      String llapUser = RegistryUtils.currentUser();
       LOG.info("Number of splits: " + (eventList.size() - 1));
       for (int i = 0; i < eventList.size() - 1; i++) {
 
@@ -355,7 +357,7 @@ public class GenericUDTFGetSplits extends GenericUDTF {
 
         byte[] submitWorkBytes = SubmitWorkInfo.toBytes(submitWorkInfo);
 
-        result[i] = new LlapInputSplit(i, submitWorkBytes, dob.getData(), locations, schema);
+        result[i] = new LlapInputSplit(i, submitWorkBytes, dob.getData(), locations, schema, llapUser);
       }
       return result;
     } catch (Exception e) {


[12/39] hive git commit: HIVE-13368: LlapTaskUmbilicalExternalClient should handle submission rejection/failures/timeouts from LLAP daemon

Posted by jd...@apache.org.
HIVE-13368: LlapTaskUmbilicalExternalClient should handle submission rejection/failures/timeouts from LLAP daemon


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/134f2f74
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/134f2f74
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/134f2f74

Branch: refs/heads/master
Commit: 134f2f749e1981738a660843e911db173c86bcfa
Parents: 2514065
Author: Jason Dere <jd...@hortonworks.com>
Authored: Mon Mar 28 11:16:33 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Mon Mar 28 11:16:33 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/llap/LlapInputFormat.java       | 137 ++++++++++-
 .../ext/LlapTaskUmbilicalExternalClient.java    | 237 ++++++++++++++++++-
 .../hadoop/hive/llap/LlapRecordReader.java      | 123 +++++++++-
 3 files changed, 480 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/134f2f74/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
index 847c74f..7f11e1d 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/LlapInputFormat.java
@@ -23,13 +23,18 @@ import java.net.Socket;
 import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
 
 import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
+
+import org.apache.commons.collections4.ListUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.llap.LlapRecordReader.ReaderEvent;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
 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;
@@ -52,11 +57,19 @@ 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.runtime.api.events.TaskAttemptFailedEvent;
 import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.apache.tez.runtime.api.impl.TezEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.tez.dag.records.TezTaskAttemptID;
+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;
+
+
 public class LlapInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
 
   private static final Logger LOG = LoggerFactory.getLogger(LlapInputFormat.class);
@@ -84,9 +97,11 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     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());
+          submitWorkInfo.getToken(), umbilicalResponder);
     llapClient.init(job);
     llapClient.start();
 
@@ -117,7 +132,9 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
 
     LOG.info("Registered id: " + id);
 
-    return new LlapRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class);
+    LlapRecordReader recordReader = new LlapRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class);
+    umbilicalResponder.setRecordReader(recordReader);
+    return recordReader;
   }
 
   @Override
@@ -254,4 +271,120 @@ public class LlapInputFormat<V extends WritableComparable> implements InputForma
     containerCredentials.writeTokenStorageToStream(containerTokens_dob);
     return ByteBuffer.wrap(containerTokens_dob.getData(), 0, containerTokens_dob.getLength());
   }
+
+  private static class LlapRecordReaderTaskUmbilicalExternalResponder implements LlapTaskUmbilicalExternalResponder {
+    protected LlapRecordReader recordReader = null;
+    protected LinkedBlockingQueue<ReaderEvent> queuedEvents = new LinkedBlockingQueue<ReaderEvent>();
+
+    public LlapRecordReaderTaskUmbilicalExternalResponder() {
+    }
+
+    @Override
+    public void submissionFailed(String fragmentId, Throwable throwable) {
+      try {
+        sendOrQueueEvent(LlapRecordReader.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(LlapRecordReader.ReaderEvent.doneEvent());
+              break;
+            case TASK_ATTEMPT_FAILED_EVENT:
+              TaskAttemptFailedEvent taskFailedEvent = (TaskAttemptFailedEvent) tezEvent.getEvent();
+              sendOrQueueEvent(LlapRecordReader.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(LlapRecordReader.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(LlapRecordReader.ReaderEvent.errorEvent(
+            "Timed out waiting for heartbeat for task ID " + taskAttemptId));
+      } catch (Exception err) {
+        LOG.error("Error during heartbeat responder:", err);
+      }
+    }
+
+    public synchronized LlapRecordReader getRecordReader() {
+      return recordReader;
+    }
+
+    public synchronized void setRecordReader(LlapRecordReader 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()) {
+        LlapRecordReader.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(LlapRecordReader.ReaderEvent readerEvent) {
+      LlapRecordReader 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/134f2f74/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java b/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
index 16cfd01..7d06637 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
@@ -2,12 +2,17 @@ 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;
@@ -19,16 +24,20 @@ 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);
@@ -41,20 +50,51 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
   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 final ConcurrentMap<String, List<TezEvent>> pendingEvents = new ConcurrentHashMap<>();
+  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) {
+  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);
@@ -71,6 +111,7 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
   @Override
   public void serviceStop() {
     llapTaskUmbilicalServer.shutdownServer();
+    timer.shutdown();
     if (this.communicator != null) {
       this.communicator.stop();
     }
@@ -89,7 +130,15 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
     Preconditions.checkArgument(submitWorkRequestProto.getUsingTezAm() == false);
 
     // Register the pending events to be sent for this spec.
-    pendingEvents.putIfAbsent(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), tezEvents);
+    String fragmentId = submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString();
+    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,
@@ -99,7 +148,12 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
           public void setResponse(LlapDaemonProtocolProtos.SubmitWorkResponseProto response) {
             if (response.hasSubmissionState()) {
               if (response.getSubmissionState().equals(LlapDaemonProtocolProtos.SubmissionStateProto.REJECTED)) {
-                LOG.info("Fragment: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString() + " rejected. Server Busy.");
+                String msg = "Fragment: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString() + " rejected. Server Busy.";
+                LOG.info(msg);
+                if (responder != null) {
+                  Throwable err = new RuntimeException(msg);
+                  responder.submissionFailed(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), err);
+                }
                 return;
               }
             }
@@ -107,7 +161,10 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
 
           @Override
           public void indicateError(Throwable t) {
-            LOG.error("Failed to submit: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), t);
+            String msg = "Failed to submit: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString();
+            LOG.error(msg, t);
+            Throwable err = new RuntimeException(msg, t);
+            responder.submissionFailed(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), err);
           }
         });
 
@@ -130,9 +187,101 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
 
   }
 
+  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);
+  }
 
 
 
@@ -153,16 +302,35 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
 
       // This also provides completion information, and a possible notification when task actually starts running (first heartbeat)
 
-      // Incoming events can be ignored until the point when shuffle needs to be handled, instead of just scans.
-
+      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 = pendingEvents.remove(taskAttemptId.toString());
-      if (tezEvents == null) {
+      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());
@@ -172,20 +340,63 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
       response.setNextPreRoutedEventId(0); //Irrelevant. See comment above.
       response.setEvents(tezEvents);
 
-      // TODO KKK: Should ideally handle things like Task success notifications.
-      // Can this somehow be hooked into the LlapTaskCommunicator to make testing easy
+      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 {
-      // TODO Eventually implement - to handle keep-alive messages from pending work.
+      updateHeartbeatInfo(hostname.toString(), port);
+      // No need to propagate to this to the responder
     }
 
     @Override
     public void taskKilled(TezTaskAttemptID taskAttemptId) throws IOException {
-      // TODO Eventually implement - to handle preemptions within LLAP daemons.
+      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

http://git-wip-us.apache.org/repos/asf/hive/blob/134f2f74/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
index ce3d39a..30ed9cf 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapRecordReader.java
@@ -18,9 +18,11 @@
 
 package org.apache.hadoop.hive.llap;
 
+import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.DataInputStream;
+import java.util.concurrent.LinkedBlockingQueue;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.io.RCFile.Reader;
@@ -33,16 +35,25 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.hive.metastore.api.Schema;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 public class LlapRecordReader<V extends WritableComparable> implements RecordReader<NullWritable, V> {
+  private static final Logger LOG = LoggerFactory.getLogger(LlapRecordReader.class);
 
   DataInputStream din;
   Schema schema;
   Class<V> clazz;
 
+
+  protected Thread readerThread = null;
+  protected LinkedBlockingQueue<ReaderEvent> readerEvents = new LinkedBlockingQueue<ReaderEvent>();
+
   public LlapRecordReader(InputStream in, Schema schema, Class<V> clazz) {
     din = new DataInputStream(in);
     this.schema = schema;
     this.clazz = clazz;
+    this.readerThread = Thread.currentThread();
   }
 
   public Schema getSchema() {
@@ -75,12 +86,120 @@ public class LlapRecordReader<V extends WritableComparable> implements RecordRea
   }
 
   @Override
-  public boolean next(NullWritable key, V value) {
+  public boolean next(NullWritable key, V value) throws IOException {
     try {
+      // Need a way to know what thread to interrupt, since this is a blocking thread.
+      setReaderThread(Thread.currentThread());
+
       value.readFields(din);
       return true;
-    } catch (IOException io) {
+    } catch (EOFException eof) {
+      // End of input. There should be a reader event available, or coming soon, so okay to be blocking call.
+      ReaderEvent event = getReaderEvent();
+      switch (event.getEventType()) {
+        case DONE:
+          break;
+        default:
+          throw new IOException("Expected reader event with done status, but got "
+              + event.getEventType() + " with message " + event.getMessage());
+      }
       return false;
+    } catch (IOException io) {
+      if (Thread.interrupted()) {
+        // Either we were interrupted by one of:
+        // 1. handleEvent(), in which case there is a reader event waiting for us in the queue
+        // 2. Some other unrelated cause which interrupted us, in which case there may not be a reader event coming.
+        // Either way we should not try to block trying to read the reader events queue.
+        if (readerEvents.isEmpty()) {
+          // Case 2.
+          throw io;
+        } else {
+          // Case 1. Fail the reader, sending back the error we received from the reader event.
+          ReaderEvent event = getReaderEvent();
+          switch (event.getEventType()) {
+            case ERROR:
+              throw new IOException("Received reader event error: " + event.getMessage());
+            default:
+              throw new IOException("Got reader event type " + event.getEventType() + ", expected error event");
+          }
+        }
+      } else {
+        // If we weren't interrupted, just propagate the error
+        throw io;
+      }
+    }
+  }
+
+  /**
+   * Define success/error events which are passed to the reader from a different thread.
+   * The reader will check for these events on end of input and interruption of the reader thread.
+   */
+  public static class ReaderEvent {
+    public enum EventType {
+      DONE,
+      ERROR
+    }
+
+    protected final EventType eventType;
+    protected final String message;
+
+    protected ReaderEvent(EventType type, String message) {
+      this.eventType = type;
+      this.message = message;
+    }
+
+    public static ReaderEvent doneEvent() {
+      return new ReaderEvent(EventType.DONE, "");
+    }
+
+    public static ReaderEvent errorEvent(String message) {
+      return new ReaderEvent(EventType.ERROR, message);
     }
+
+    public EventType getEventType() {
+      return eventType;
+    }
+
+    public String getMessage() {
+      return message;
+    }
+  }
+
+  public void handleEvent(ReaderEvent event) {
+    switch (event.getEventType()) {
+      case DONE:
+        // Reader will check for the event queue upon the end of the input stream - no need to interrupt.
+        readerEvents.add(event);
+        break;
+      case ERROR:
+        readerEvents.add(event);
+        if (readerThread == null) {
+          throw new RuntimeException("Reader thread is unexpectedly null, during ReaderEvent error " + event.getMessage());
+        }
+        // Reader is using a blocking socket .. interrupt it.
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Interrupting reader thread due to reader event with error " + event.getMessage());
+        }
+        getReaderThread().interrupt();
+      default:
+        throw new RuntimeException("Unhandled ReaderEvent type " + event.getEventType() + " with message " + event.getMessage());
+    }
+  }
+
+  protected ReaderEvent getReaderEvent() {
+    try {
+      ReaderEvent event = readerEvents.take();
+      return event;
+    } catch (InterruptedException ie) {
+      throw new RuntimeException("Interrupted while getting readerEvents, not expected", ie);
+    }
+  }
+
+  protected synchronized void setReaderThread(Thread readerThread) {
+    this.readerThread = readerThread;
+  }
+
+  protected synchronized Thread getReaderThread() {
+    return readerThread;
   }
 }


[23/39] hive git commit: Merge remote-tracking branch 'origin/master' into llap

Posted by jd...@apache.org.
Merge remote-tracking branch 'origin/master' into llap


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e69bd1ee
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e69bd1ee
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e69bd1ee

Branch: refs/heads/master
Commit: e69bd1ee53ba8f7aef8614d1cd3fb339e07db984
Parents: 5816ff3 9b5eb45
Author: Jason Dere <jd...@hortonworks.com>
Authored: Wed Apr 20 13:04:26 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Wed Apr 20 13:04:26 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |   85 +-
 .../test/resources/testconfiguration.properties |    3 +-
 .../apache/hive/jdbc/HiveQueryResultSet.java    |    9 +
 .../org/apache/hive/jdbc/HiveStatement.java     |   75 +-
 .../llap/io/decode/OrcEncodedDataConsumer.java  |   45 +-
 metastore/if/hive_metastore.thrift              |   59 +
 .../upgrade/derby/034-HIVE-13076.derby.sql      |    3 +
 .../upgrade/derby/hive-schema-2.1.0.derby.sql   |    6 +
 .../derby/upgrade-2.0.0-to-2.1.0.derby.sql      |    1 +
 .../upgrade/mssql/019-HIVE-13076.mssql.sql      |   15 +
 .../upgrade/mssql/hive-schema-2.1.0.mssql.sql   |   18 +
 .../mssql/upgrade-2.0.0-to-2.1.0.mssql.sql      |    1 +
 .../upgrade/mysql/034-HIVE-13076.mysql.sql      |   17 +
 .../upgrade/mysql/hive-schema-2.1.0.mysql.sql   |   17 +-
 .../mysql/upgrade-2.0.0-to-2.1.0.mysql.sql      |    1 +
 .../upgrade/oracle/034-HIVE-13076.oracle.sql    |   15 +
 .../upgrade/oracle/hive-schema-2.1.0.oracle.sql |   19 +
 .../oracle/upgrade-2.0.0-to-2.1.0.oracle.sql    |    1 +
 .../postgres/033-HIVE-13076.postgres.sql        |   15 +
 .../postgres/hive-schema-2.1.0.postgres.sql     |   17 +
 .../upgrade-2.0.0-to-2.1.0.postgres.sql         |    1 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 4885 +++++++-----
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  447 ++
 .../ThriftHiveMetastore_server.skeleton.cpp     |   15 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 6512 +++++++++-------
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  404 +
 .../metastore/api/AddDynamicPartitions.java     |   32 +-
 .../metastore/api/AddPartitionsRequest.java     |   36 +-
 .../hive/metastore/api/AddPartitionsResult.java |   36 +-
 .../metastore/api/ClearFileMetadataRequest.java |   32 +-
 .../metastore/api/DropPartitionsResult.java     |   36 +-
 .../hive/metastore/api/FireEventRequest.java    |   32 +-
 .../hive/metastore/api/ForeignKeysRequest.java  |  692 ++
 .../hive/metastore/api/ForeignKeysResponse.java |  443 ++
 .../hadoop/hive/metastore/api/Function.java     |   36 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../api/GetFileMetadataByExprRequest.java       |   32 +-
 .../api/GetFileMetadataByExprResult.java        |   48 +-
 .../metastore/api/GetFileMetadataRequest.java   |   32 +-
 .../metastore/api/GetFileMetadataResult.java    |   44 +-
 .../metastore/api/GetOpenTxnsInfoResponse.java  |   36 +-
 .../hive/metastore/api/GetOpenTxnsResponse.java |   32 +-
 .../api/HeartbeatTxnRangeResponse.java          |   64 +-
 .../metastore/api/InsertEventRequestData.java   |   32 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |   36 +-
 .../api/NotificationEventResponse.java          |   36 +-
 .../hive/metastore/api/OpenTxnsResponse.java    |   32 +-
 .../metastore/api/PartitionsByExprResult.java   |   36 +-
 .../metastore/api/PartitionsStatsRequest.java   |   64 +-
 .../metastore/api/PartitionsStatsResult.java    |   76 +-
 .../hive/metastore/api/PrimaryKeysRequest.java  |  490 ++
 .../hive/metastore/api/PrimaryKeysResponse.java |  443 ++
 .../metastore/api/PutFileMetadataRequest.java   |   64 +-
 .../hive/metastore/api/RequestPartsSpec.java    |   68 +-
 .../hive/metastore/api/SQLForeignKey.java       | 1715 +++++
 .../hive/metastore/api/SQLPrimaryKey.java       | 1103 +++
 .../hive/metastore/api/ShowCompactResponse.java |   36 +-
 .../hive/metastore/api/ShowLocksResponse.java   |   36 +-
 .../hive/metastore/api/TableStatsRequest.java   |   32 +-
 .../hive/metastore/api/TableStatsResult.java    |   36 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 7122 +++++++++++++-----
 .../gen-php/metastore/ThriftHiveMetastore.php   | 2260 ++++--
 .../src/gen/thrift/gen-php/metastore/Types.php  | 5564 ++++++++------
 .../hive_metastore/ThriftHiveMetastore-remote   |   21 +
 .../hive_metastore/ThriftHiveMetastore.py       | 1890 +++--
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  | 1440 +++-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |  152 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |  209 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   97 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   46 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   16 +
 .../hive/metastore/MetaStoreDirectSql.java      |  134 +
 .../hadoop/hive/metastore/ObjectStore.java      |  397 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |   12 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |   24 +
 .../hive/metastore/model/MConstraint.java       |  148 +
 metastore/src/model/package.jdo                 |   33 +
 .../DummyRawStoreControlledCommit.java          |   24 +
 .../DummyRawStoreForJdoConnection.java          |   24 +
 orc/src/java/org/apache/orc/DataReader.java     |    6 +-
 .../java/org/apache/orc/DataReaderFactory.java  |    9 +
 .../org/apache/orc/MetadataReaderFactory.java   |   12 +
 orc/src/java/org/apache/orc/OrcUtils.java       |   75 -
 orc/src/java/org/apache/orc/Reader.java         |    6 -
 orc/src/java/org/apache/orc/RecordReader.java   |    8 +-
 .../java/org/apache/orc/TypeDescription.java    |   62 +-
 .../org/apache/orc/impl/BitFieldReader.java     |    5 +-
 .../apache/orc/impl/DataReaderProperties.java   |   84 +
 .../orc/impl/DefaultMetadataReaderFactory.java  |   14 +
 .../java/org/apache/orc/impl/IntegerReader.java |   26 +-
 .../org/apache/orc/impl/MetadataReader.java     |    5 +-
 .../org/apache/orc/impl/MetadataReaderImpl.java |   17 +-
 .../orc/impl/MetadataReaderProperties.java      |   96 +
 .../apache/orc/impl/RunLengthByteReader.java    |   36 +-
 .../apache/orc/impl/RunLengthIntegerReader.java |   31 +-
 .../orc/impl/RunLengthIntegerReaderV2.java      |   33 +-
 .../java/org/apache/orc/impl/WriterImpl.java    |   47 +-
 .../orc/impl/TestDataReaderProperties.java      |   69 +
 .../orc/impl/TestMetadataReaderProperties.java  |   72 +
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   42 +-
 .../ql/exec/vector/VectorizedRowBatchCtx.java   |   13 +-
 .../ql/io/orc/DefaultDataReaderFactory.java     |   14 +
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   43 +-
 .../hive/ql/io/orc/OrcRawRecordMerger.java      |    3 +-
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |   50 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |  203 +-
 .../hive/ql/io/orc/RecordReaderUtils.java       |   17 +-
 .../hadoop/hive/ql/io/orc/SchemaEvolution.java  |  234 +-
 .../hive/ql/io/orc/TreeReaderFactory.java       |  838 ++-
 .../ql/io/orc/VectorizedOrcInputFormat.java     |   32 +-
 .../hadoop/hive/ql/io/orc/WriterImpl.java       |    2 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    7 +-
 .../hadoop/hive/ql/session/SessionState.java    |    2 +-
 .../hive/ql/io/orc/TestRecordReaderImpl.java    |  145 +-
 .../hive/ql/io/orc/TestTypeDescription.java     |    4 +-
 .../hive/ql/io/orc/TestVectorOrcFile.java       | 1634 ++--
 .../hive/ql/io/orc/TestVectorizedORCReader.java |    7 +-
 .../queries/clientpositive/encryption_ctas.q    |   16 +
 .../encrypted/encryption_ctas.q.out             |   56 +
 .../gen-py/hive_service/ThriftHive-remote       |   21 +
 .../cli/operation/GetCatalogsOperation.java     |    5 +-
 .../cli/operation/GetColumnsOperation.java      |    5 +-
 .../cli/operation/GetFunctionsOperation.java    |    6 +-
 .../cli/operation/GetSchemasOperation.java      |    7 +-
 .../cli/operation/GetTableTypesOperation.java   |    7 +-
 .../cli/operation/GetTablesOperation.java       |   10 +-
 .../cli/operation/GetTypeInfoOperation.java     |    7 +-
 .../hive/service/cli/operation/Operation.java   |    8 +-
 .../service/cli/operation/SQLOperation.java     |    7 +-
 .../hive/ql/exec/vector/BytesColumnVector.java  |   11 -
 .../ql/exec/vector/TimestampColumnVector.java   |    2 +-
 .../hive/ql/exec/vector/UnionColumnVector.java  |    2 +
 .../ptest2/src/main/resources/source-prep.vm    |    2 -
 133 files changed, 30015 insertions(+), 12493 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e69bd1ee/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------