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 ] [-u ] [-p ] " + echo "" + echo " --location (-l) hs2 url" + echo " --user (-u) user name" + echo " --pwd (-p) password" +} 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 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 06a6906..d52d629 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -1033,7 +1033,7 @@ private static void populateLlapDaemonVarsSet(Set llapDaemonVarsSetLocal "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"), @@ -2670,6 +2670,9 @@ private static void populateLlapDaemonVarsSet(Set llapDaemonVarsSetLocal "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. @@ -2830,6 +2833,8 @@ private static void populateLlapDaemonVarsSet(Set llapDaemonVarsSetLocal 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), 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 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 @@ org.apache.hive + hive-llap-ext-client + ${project.version} + + + org.apache.hive hive-llap-server ${project.version} test-jar 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..1de8aa6 --- /dev/null +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java @@ -0,0 +1,102 @@ +/** + * 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; +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.LlapInputSplit; +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 colDescs = new ArrayList(); + 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); + + LlapInputSplit split1 = new 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); + LlapInputSplit split2 = new LlapInputSplit(); + split2.readFields(dataIn); + + // Did we read all the data? + assertEquals(0, byteInStream.available()); + + checkLlapSplits(split1, split2); + } + + static void checkLlapSplits( + LlapInputSplit split1, + 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()); + } + +} 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..48b9493 --- /dev/null +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlap.java @@ -0,0 +1,234 @@ +/** + * 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.Timer; +import java.util.TimerTask; +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.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.hadoop.hive.llap.LlapBaseInputFormat; +import org.apache.hadoop.hive.llap.LlapRowInputFormat; + +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 static HiveConf conf = null; + 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()); + } + + 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")); + + miniHS2 = new MiniHS2(conf, MiniClusterType.LLAP); + + dataFileDir = conf.get("test.data.files").replace('\\', '/').replace("c:", ""); + kvDataFilePath = new Path(dataFileDir, "kv1.txt"); + Map confOverlay = new HashMap(); + 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(); + } + + private int getLlapIFRowCount(String query, int numSplits) throws Exception { + + // Setup LlapInputFormat + String url = miniHS2.getJdbcURL(); + String user = System.getProperty("user.name"); + String pwd = user; + + 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); + + // Fetch rows from splits + boolean first = true; + int rowCount = 0; + for (InputSplit split : splits) { + System.out.println("Processing split " + split.getLocations()); + + int numColumns = 2; + RecordReader 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) { + System.out.println("Results: "); + System.out.println(""); + first = false; + } + + 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; + } + } + + return rowCount; + } + + @Test(timeout = 60000) + 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 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 @@ + + org.apache.tez + tez-api + ${tez.version} + true + + + org.slf4j + slf4j-log4j12 + + + commmons-logging + commons-logging + + + + + org.apache.tez + tez-runtime-internals + ${tez.version} + true + + + org.slf4j + slf4j-log4j12 + + + commmons-logging + commons-logging + + + ${basedir}/src/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..3c858a8 --- /dev/null +++ b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapBaseRecordReader.java @@ -0,0 +1,221 @@ +/** + * 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 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; +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; + +/** + * Base LLAP RecordReader to handle receiving of the data from the LLAP daemon. + */ +public class LlapBaseRecordReader implements RecordReader { + private static final Logger LOG = LoggerFactory.getLogger(LlapBaseRecordReader.class); + + protected final DataInputStream din; + protected final Schema schema; + protected final Class clazz; + + protected Thread readerThread = null; + protected final LinkedBlockingQueue readerEvents = new LinkedBlockingQueue(); + protected final long timeout; + + public LlapBaseRecordReader(InputStream in, Schema schema, Class 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() { + return schema; + } + + @Override + public void close() throws IOException { + din.close(); + } + + @Override + public long getPos() { + // dummy impl + return 0; + } + + @Override + public float getProgress() { + // dummy impl + 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 (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()) { + // 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() throws IOException { + try { + 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); + } + } + + protected synchronized void setReaderThread(Thread readerThread) { + this.readerThread = readerThread; + } + + protected synchronized Thread getReaderThread() { + return readerThread; + } +} 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; + } +} 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..084da0a --- /dev/null +++ b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapRowRecordReader.java @@ -0,0 +1,173 @@ +/** + * 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; + +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; + +/** + * Row-based record reader for LLAP. + */ +public class LlapRowRecordReader implements RecordReader { + + private static final Logger LOG = LoggerFactory.getLogger(LlapRowRecordReader.class); + + protected final Configuration conf; + protected final RecordReader reader; + protected final Schema schema; + protected final SerDe serde; + protected final Text textData = new Text(); + + public LlapRowRecordReader(Configuration conf, Schema schema, RecordReader reader) throws IOException { + this.conf = conf; + this.schema = schema; + this.reader = reader; + + try { + serde = initSerDe(conf); + } catch (SerDeException err) { + throw new IOException(err); + } + } + + @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); + + 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 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; + } +} 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..6704294 --- /dev/null +++ b/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java @@ -0,0 +1,119 @@ +/* + * 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.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 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 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 createJobToken() { + String tokenIdentifier = fakeAppId.toString(); + JobTokenIdentifier identifier = new JobTokenIdentifier(new Text( + tokenIdentifier)); + Token sessionToken = new Token(identifier, + new JobTokenSecretManager()); + sessionToken.setService(identifier.getJobId()); + return sessionToken; + } +} 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..0edb1cd --- /dev/null +++ b/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java @@ -0,0 +1,413 @@ +/* + * 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; +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 sessionToken; + + private final ConcurrentMap pendingEvents = new ConcurrentHashMap<>(); + private final ConcurrentMap registeredTasks= new ConcurrentHashMap(); + 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 tezEvents; + + public PendingEventData(TaskHeartbeatInfo heartbeatInfo, List tezEvents) { + this.heartbeatInfo = heartbeatInfo; + this.tezEvents = tezEvents; + } + } + + public LlapTaskUmbilicalExternalClient(Configuration conf, String tokenIdentifier, + Token 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 = 3 * HiveConf.getTimeVar(conf, + HiveConf.ConfVars.LLAP_DAEMON_AM_LIVENESS_CONNECTION_TIMEOUT_MS, TimeUnit.MILLISECONDS); + // 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 tezEvents) { + // Register the pending events to be sent for this spec. + 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() { + + @Override + public void setResponse(LlapDaemonProtocolProtos.SubmitWorkResponseProto response) { + if (response.hasSubmissionState()) { + if (response.getSubmissionState().equals(LlapDaemonProtocolProtos.SubmissionStateProto.REJECTED)) { + String msg = "Fragment: " + fragmentId + " rejected. Server Busy."; + LOG.info(msg); + if (responder != null) { + Throwable err = new RuntimeException(msg); + responder.submissionFailed(fragmentId, err); + } + return; + } + } + } + + @Override + public void indicateError(Throwable t) { + String msg = "Failed to submit: " + fragmentId; + LOG.error(msg, t); + Throwable err = new RuntimeException(msg, t); + responder.submissionFailed(fragmentId, err); + } + }); + } + + 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 timedOutTasks = new ArrayList(); + + // 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); + } + + 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); + } + } + } + + public interface LlapTaskUmbilicalExternalResponder { + void submissionFailed(String fragmentId, Throwable throwable); + void heartbeat(TezHeartbeatRequest request); + void taskKilled(TezTaskAttemptID taskAttemptId); + void heartbeatTimeout(String fragmentId); + } + + + + // 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 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 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); + } + } + +} 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 7e37e96..9004d3c 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 @@ -60,6 +60,13 @@ */ public String getServicesAddress(); /** + * OutputFormat endpoint for service instance + * + * @return + */ + public int getOutputFormatPort(); + + /** * Return the last known state (without refreshing) * * @return 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 67443a7..e9456f2 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 @@ -55,6 +55,7 @@ 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; @@ -69,6 +70,7 @@ public LlapFixedRegistryImpl(String hosts, Configuration conf) { 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); @@ -171,6 +173,11 @@ public int getShufflePort() { } @Override + public int getOutputFormatPort() { + return LlapFixedRegistryImpl.this.outputFormatPort; + } + + @Override public String getServicesAddress() { return serviceAddress; } 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 6981061..fde70e7 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 @@ -89,6 +89,7 @@ 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-"; @@ -177,7 +178,7 @@ public LlapZookeeperRegistryImpl(String instanceName, Configuration conf) { // 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.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; @@ -212,6 +213,13 @@ private String getQuorumServers(Configuration conf) { 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)); @@ -244,6 +252,11 @@ public Endpoint getMngEndpoint() { 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 String register() throws IOException { ServiceRecord srv = new ServiceRecord(); @@ -252,6 +265,7 @@ public String register() throws IOException { srv.addInternalEndpoint(getMngEndpoint()); srv.addInternalEndpoint(getShuffleEndpoint()); srv.addExternalEndpoint(getServicesEndpoint()); + srv.addInternalEndpoint(getOutputFormatEndpoint()); for (Map.Entry kv : this.conf) { if (kv.getKey().startsWith(HiveConf.PREFIX_LLAP) @@ -343,6 +357,7 @@ public void unregister() throws IOException { 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 { @@ -355,6 +370,7 @@ public DynamicServiceInstance(ServiceRecord srv) throws IOException { 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 = @@ -369,6 +385,9 @@ public DynamicServiceInstance(ServiceRecord srv) throws IOException { this.shufflePort = 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); } @@ -433,6 +452,11 @@ public int getManagementPort() { 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. } 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..79800da --- /dev/null +++ b/llap-client/src/java/org/apache/hadoop/hive/llap/tezplugins/helpers/LlapTaskUmbilicalServer.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.llap.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 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(); + } + } +} 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 @@ hive-common ${project.version} + + org.apache.hive + hive-serde + ${project.version} + @@ -58,6 +63,22 @@ + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + true + + + org.slf4j + slf4j-log4j12 + + + commmons-logging + commons-logging + + + + org.apache.tez tez-api ${tez.version} 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 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 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(schema.getColumns().size()); + + List 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()); + } + } +} 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 columns; + + public Schema(List columns) { + this.columns = columns; + } + + public Schema() { + columns = new ArrayList(); + } + + public List 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); + } + } +} 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 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 colDescs = new ArrayList(); + + 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 diff --git a/llap-ext-client/pom.xml b/llap-ext-client/pom.xml new file mode 100644 index 0000000..fdf16cd --- /dev/null +++ b/llap-ext-client/pom.xml @@ -0,0 +1,173 @@ + + + + 4.0.0 + + org.apache.hive + hive + 2.1.0-SNAPSHOT + ../pom.xml + + + hive-llap-ext-client + jar + Hive Llap External Client + + + .. + + + + + + + org.apache.hive + hive-jdbc + ${project.version} + + + org.apache.hive + hive-llap-client + ${project.version} + + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + true + + + org.slf4j + slf4j-log4j12 + + + commmons-logging + commons-logging + + + + + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + true + + + org.apache.hadoop + hadoop-yarn-registry + ${hadoop.version} + true + + + org.apache.tez + tez-api + ${tez.version} + true + + + org.slf4j + slf4j-log4j12 + + + commmons-logging + commons-logging + + + + + org.apache.tez + tez-runtime-internals + ${tez.version} + true + + + org.slf4j + slf4j-log4j12 + + + commmons-logging + commons-logging + + + + + + + junit + junit + ${junit.version} + test + + + org.apache.commons + commons-lang3 + ${commons-lang3.version} + test + + + org.mockito + mockito-all + ${mockito-all.version} + test + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + tests + test + + + org.slf4j + slf4j-log4j12 + + + commmons-logging + commons-logging + + + + + + ${basedir}/src/java + ${basedir}/src/test + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-source + generate-sources + + add-source + + + + src/gen/protobuf/gen-java + src/gen/thrift/gen-javabean + + + + + + + + + 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..4306c22 --- /dev/null +++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java @@ -0,0 +1,468 @@ +/* + * 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; + + +/** + * Base LLAP input format to handle requesting of splits and communication with LLAP daemon. + */ +public class LlapBaseInputFormat implements InputFormat { + + 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 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 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, job); + umbilicalResponder.setRecordReader(recordReader); + return recordReader; + } + + @Override + public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { + List ins = new ArrayList(); + + 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 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 token) throws + IOException { + TaskSpec taskSpec = submitWorkInfo.getTaskSpec(); + ApplicationId appId = submitWorkInfo.getFakeAppId(); + + int attemptId = taskSpec.getTaskAttemptID().getId(); + // This works, assuming the executor is running within YARN. + String user = System.getenv(ApplicationConstants.Environment.USER.name()); + LOG.info("Setting user in submitWorkRequest to: " + user); + SignableVertexSpec svs = Converters.convertTaskSpecToProto( + taskSpec, attemptId, appId.toString(), null, user); // TODO signatureKeyId + + ContainerId containerId = + ContainerId.newInstance(ApplicationAttemptId.newInstance(appId, 0), taskNum); + + + Credentials taskCredentials = new Credentials(); + // Credentials can change across DAGs. Ideally construct only once per DAG. + Credentials credentials = new Credentials(); + TokenCache.setSessionToken(token, credentials); + ByteBuffer credentialsBinary = serializeCredentials(credentials); + + 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 queuedEvents = new LinkedBlockingQueue(); + + 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 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(); + } + } +} 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..08ad1f5 --- /dev/null +++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapDump.java @@ -0,0 +1,215 @@ +/** + * 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.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; + +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.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.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; + +/** + * 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); + + private static String url = "jdbc:hive2://localhost:10000/default"; + private static String user = "hive"; + private static String pwd = ""; + private static String query = null; + 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("llapdump", 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"); + } + + Properties configProps = cli.getOptionProperties("hiveconf"); + + if (cli.getArgs().length > 0) { + 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); + + 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)); + + 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 reader = format.getRecordReader(s, job, null); + + if (reader instanceof LlapRowRecordReader && first) { + Schema schema = ((LlapRowRecordReader)reader).getSchema(); + System.out.println(""+schema); + } + + if (first) { + System.out.println("Results: "); + System.out.println(""); + first = false; + } + + Row value = reader.createValue(); + while (reader.next(NullWritable.get(), 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(); + + 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')); + + 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; + } +} 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..7efc711 --- /dev/null +++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java @@ -0,0 +1,54 @@ +/** + * 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.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; + +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 { + LlapBaseInputFormat baseInputFormat = new LlapBaseInputFormat(); + + @Override + public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { + return baseInputFormat.getSplits(job, numSplits); + } + + @Override + public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) + throws IOException { + LlapInputSplit llapSplit = (LlapInputSplit) split; + LlapBaseRecordReader reader = (LlapBaseRecordReader) baseInputFormat.getRecordReader(llapSplit, job, reporter); + return new LlapRowRecordReader(job, reader.getSchema(), reader); + } +} 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 getInputFormatClass() { + throw new RuntimeException("Should not be called."); + } + + @Override + public Class getOutputFormatClass() { + return LlapOutputFormat.class; + } +} 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 2bfe3ed..2524dc2 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 @@ -272,7 +272,7 @@ public QueryCompleteResponseProto queryComplete( 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()); } 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 d23a44a..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 @@ -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; @@ -342,6 +343,7 @@ public void serviceStart() throws Exception { if (webServices != null) { getConfig().setInt(ConfVars.LLAP_DAEMON_WEB_PORT.varname, webServices.getPort()); } + getConfig().setInt(ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT.varname, LlapOutputFormatService.get().getPort()); this.registry.init(getConfig()); this.registry.start(); @@ -358,6 +360,7 @@ public void serviceStop() throws Exception { super.serviceStop(); ShuffleHandler.shutdown(); shutdown(); + LlapOutputFormatService.get().stop(); LOG.info("LlapDaemon shutdown complete"); } 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 3093de7..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,9 +141,9 @@ public TaskRunnerCallable(SubmitWorkRequestProto request, QueryFragmentInfo frag 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(), - 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(); 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 static SubmitWorkRequestProto createSubmitWorkRequestProto( 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); 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 a3f2eb8..026df3b 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 @@ -355,7 +355,7 @@ public void unregisterRunningTaskAttempt(final TezTaskAttemptID taskAttemptId, 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 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 da1e17f..a501f7d 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 @@ -1121,7 +1121,7 @@ private void preemptTasks(int forPriority, int numTasksToPreempt, String []poten } } 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; } } @@ -1131,7 +1131,7 @@ private void preemptTasks(int forPriority, int numTasksToPreempt, String []poten // 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. diff --git a/pom.xml b/pom.xml index dff2a72..ab925de 100644 --- a/pom.xml +++ b/pom.xml @@ -50,6 +50,7 @@ service llap-common llap-client + llap-ext-client llap-tez llap-server shims 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(); + } + } +} 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 + implements OutputFormat { + + public static final String LLAP_OF_ID_KEY = "llap.of.id"; + + @Override + public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException { + } + + @Override + public RecordWriter 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().getWriter(job.get(LLAP_OF_ID_KEY)); + } catch (InterruptedException e) { + throw new IOException(e); + } + } +} 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..6adbf7c --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java @@ -0,0 +1,196 @@ +/* + * 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.io.OutputStream; +import java.net.InetSocketAddress; + +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 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; + + +/** + * Responsible for sending back result set data to the connections made by external clients via the LLAP input format. + */ +public class LlapOutputFormatService { + + private static final Logger LOG = LoggerFactory.getLogger(LlapOutputFormat.class); + + private static LlapOutputFormatService service; + private final Map writers; + private final HiveConf conf; + 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 int port; + + private LlapOutputFormatService() throws IOException { + writers = new HashMap(); + conf = new HiveConf(); + } + + public static LlapOutputFormatService get() throws IOException { + if (service == null) { + service = new LlapOutputFormatService(); + service.start(); + } + return service; + } + + public void start() throws IOException { + LOG.info("Starting LlapOutputFormatService"); + + 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 { + 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 " + portFromConf, err); + } + } + + public void stop() throws IOException, InterruptedException { + 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 RecordWriter getWriter(String id) throws IOException, InterruptedException { + RecordWriter writer = null; + synchronized(service) { + while ((writer = writers.get(id)) == null) { + LOG.info("Waiting for writer for: "+id); + service.wait(); + } + } + LOG.info("Returning writer for: "+id); + return writer; + } + + public int getPort() { + return port; + } + + protected class LlapOutputFormatServiceHandler extends SimpleChannelInboundHandler { + @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 { + @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()); + } + } +} 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..b632fae --- /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.io.Writable; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapred.RecordWriter; +import org.apache.hadoop.mapred.Reporter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LlapRecordWriter + implements RecordWriter { + public static final Logger LOG = LoggerFactory.getLogger(LlapRecordWriter.class); + + DataOutputStream dos; + + public LlapRecordWriter(OutputStream out) { + dos = new DataOutputStream(out); + } + + @Override + public void close(Reporter reporter) throws IOException { + LOG.info("CLOSING the record writer output stream"); + dos.close(); + } + + @Override + public void write(K key, V value) throws IOException { + value.write(dos); + } +} 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 00df3a0..2f4a94c 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 @@ -446,6 +446,7 @@ system.registerGenericUDTF("parse_url_tuple", GenericUDTFParseUrlTuple.class); system.registerGenericUDTF("posexplode", GenericUDTFPosExplode.class); system.registerGenericUDTF("stack", GenericUDTFStack.class); + system.registerGenericUDTF("get_splits", GenericUDTFGetSplits.class); //PTF declarations system.registerGenericUDF(LEAD_FUNC_NAME, GenericUDFLead.class); 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..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 @@ -33,6 +33,9 @@ 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.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,9 +86,34 @@ private final SplitGrouper splitGrouper = new SplitGrouper(); private final SplitLocationProvider splitLocationProvider; + public HiveSplitGenerator(Configuration conf, MapWork work) throws IOException { + super(null); + + this.conf = conf; + this.work = work; + this.jobConf = new JobConf(conf); + + // 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); + + // 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. + // No dynamic partition pruning + pruner = null; + } + public HiveSplitGenerator(InputInitializerContext initializerContext) throws IOException, SerDeException { super(initializerContext); + Preconditions.checkNotNull(initializerContext); userPayloadProto = MRInputHelpers.parseMRInputPayload(initializerContext.getInputUserPayload()); @@ -120,7 +149,9 @@ public HiveSplitGenerator(InputInitializerContext initializerContext) throws IOE 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); @@ -133,9 +164,20 @@ public HiveSplitGenerator(InputInitializerContext initializerContext) throws IOE (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 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 9a9f43a..dc63d7b 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 @@ -36,6 +36,7 @@ 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; @@ -95,6 +96,9 @@ public MapRecordProcessor(final JobConf jconf, final ProcessorContext context) t super(jconf, context); String queryId = HiveConf.getVar(jconf, HiveConf.ConfVars.HIVEQUERYID); if (LlapProxy.isDaemon()) { // do not cache plan + 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); 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 3789ce9..f059aea 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 @@ -409,7 +409,7 @@ DAG build(JobConf conf, TezWork work, Path scratchDir, return dag; } - private void setAccessControlsForCurrentUser(DAG dag) { + public static void setAccessControlsForCurrentUser(DAG dag) { // get current user String currentUser = SessionState.getUserFromAuthenticator(); if(LOG.isDebugEnabled()) { 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 da236d5..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 @@ -1278,13 +1278,13 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, return getXpathOrFuncExprNodeDesc(expr, isFunction, children, ctx); } catch (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) { throw new SemanticException(ErrorMsg.INVALID_ARGUMENT_LENGTH.getMsg( - expr, e.getMessage())); + expr, e.getMessage()), e); } catch (UDFArgumentException e) { throw new SemanticException(ErrorMsg.INVALID_ARGUMENT.getMsg(expr, e - .getMessage())); + .getMessage()), e); } } 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 c39a46f..9c385d1 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 @@ -45,6 +45,7 @@ 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; @@ -271,6 +272,12 @@ public static TableDesc getTableDesc( 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; 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..ce69ee6 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java @@ -0,0 +1,496 @@ +/** + * 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.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; +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.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; +import org.apache.hadoop.mapred.SplitLocationInfo; +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; +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 names = Arrays.asList("split"); + List fieldOIs = Arrays.asList( + 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[1]; + bos.reset(); + s.write(dos); + byte[] frozen = bos.toByteArray(); + os[0] = 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> roots = plan.getRootTasks(); + Schema schema = convertSchema(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 = convertSchema(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()); + + 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(), 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: + Preconditions.checkState(HiveConf.getBoolVar(wxConf, + HiveConf.ConfVars.HIVE_TEZ_GENERATE_CONSISTENT_SPLITS)); + Preconditions.checkState(HiveConf.getBoolVar(wxConf, + HiveConf.ConfVars.LLAP_CLIENT_CONSISTENT_SPLITS)); + HiveSplitGenerator splitGenerator = new HiveSplitGenerator(wxConf, mapWork); + List eventList = splitGenerator.initialize(); + + InputSplit[] result = new InputSplit[eventList.size() - 1]; + DataOutputBuffer dob = new DataOutputBuffer(); + + InputConfigureVertexTasksEvent configureEvent + = (InputConfigureVertexTasksEvent) eventList.get(0); + + List 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); + + String llapUser = UserGroupInformation.getLoginUser().getShortUserName(); + 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 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, llapUser); + } + 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(); + } + } + } + + 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.VARCHAR, 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 colDescs = new ArrayList(); + 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 { + } +} 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..6d31802 --- /dev/null +++ b/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java @@ -0,0 +1,77 @@ +/* + * 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; +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; + +// 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, int index) { + Vertex vertex = dag.getVertex(vertexName); + ProcessorDescriptor processorDescriptor = vertex.getProcessorDescriptor(); + List> inputs = + vertex.getInputs(); + List> outputs = + vertex.getOutputs(); + + Preconditions.checkState(inputs.size() == 1); + Preconditions.checkState(outputs.size() == 1); + + List inputSpecs = new ArrayList<>(); + for (RootInputLeafOutput input : inputs) { + InputSpec inputSpec = new InputSpec(input.getName(), input.getIODescriptor(), 1); + inputSpecs.add(inputSpec); + } + + List outputSpecs = new ArrayList<>(); + for (RootInputLeafOutput output : outputs) { + OutputSpec outputSpec = new OutputSpec(output.getName(), output.getIODescriptor(), 1); + outputSpecs.add(outputSpec); + } + + TezDAGID dagId = TezDAGID.getInstance(appId, 0); + TezVertexID vertexId = TezVertexID.getInstance(dagId, 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); + } + + public EventMetaData getDestingationMetaData(Vertex vertex) { + List> inputs = + vertex.getInputs(); + Preconditions.checkState(inputs.size() == 1); + String inputName = inputs.get(0).getName(); + EventMetaData destMeta = + new EventMetaData(EventMetaData.EventProducerConsumerType.INPUT, vertex.getName(), + inputName, null); + return destMeta; + } + +} 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..907d5b0 --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java @@ -0,0 +1,134 @@ +/** + * 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.LlapBaseRecordReader.ReaderEvent; +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(); + + for (int k = 0; k < 5; ++k) { + String id = "foobar"+k; + job.set(LlapOutputFormat.LLAP_OF_ID_KEY, id); + 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(id.getBytes()); + socket.getOutputStream().write(0); + socket.getOutputStream().flush(); + + Thread.sleep(3000); + + LOG.debug("Data written"); + + RecordWriter 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(); + LlapBaseRecordReader reader = new LlapBaseRecordReader(in, null, Text.class, job); + + 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()); + count++; + } + + reader.close(); + + Assert.assertEquals(count,10); + } + } +} 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