diff --git itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java index 7f78a2b..21651b9 100644 --- itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java +++ itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java @@ -124,7 +124,7 @@ public static void afterTest() throws Exception { } /** - * Verify that the connection to HS2 with MiniMr is successful + * Verify that the connection to HS2 with MiniMr is successful. * @throws Exception */ @Test @@ -134,7 +134,7 @@ public void testConnection() throws Exception { } /** - * Run nonMr query + * Run nonMr query. * @throws Exception */ @Test @@ -147,15 +147,15 @@ public void testNonSparkQuery() throws Exception { } /** - * Run nonMr query + * Run nonMr query. * @throws Exception */ @Test public void testSparkQuery() throws Exception { String tableName = "testTab2"; String resultVal = "val_238"; - String queryStr = "SELECT * FROM " + tableName + - " where value = '" + resultVal + "'"; + String queryStr = "SELECT * FROM " + tableName + + " where value = '" + resultVal + "'"; testKvQuery(tableName, queryStr, resultVal); } @@ -233,8 +233,8 @@ public void testTempTable() throws Exception { + dataFilePath.toString() + "' into table " + tempTableName); String resultVal = "val_238"; - String queryStr = "SELECT * FROM " + tempTableName + - " where value = '" + resultVal + "'"; + String queryStr = "SELECT * FROM " + tempTableName + + " where value = '" + resultVal + "'"; verifyResult(queryStr, resultVal, 2); // A second connection should not be able to see the table @@ -244,8 +244,7 @@ public void testTempTable() throws Exception { stmt2.execute("USE " + dbName); boolean gotException = false; try { - ResultSet res; - res = stmt2.executeQuery(queryStr); + stmt2.executeQuery(queryStr); } catch (SQLException err) { // This is expected to fail. assertTrue("Expecting table not found error, instead got: " + err, @@ -266,7 +265,7 @@ private void checkForNotExist(ResultSet res) throws Exception { } /** - * Verify if the given property contains the expected value + * Verify if the given property contains the expected value. * @param propertyName * @param expectedValue * @throws Exception @@ -275,7 +274,7 @@ private void verifyProperty(String propertyName, String expectedValue) throws Ex Statement stmt = hs2Conn .createStatement(); ResultSet res = stmt.executeQuery("set " + propertyName); assertTrue(res.next()); - String results[] = res.getString(1).split("="); + String[] results = res.getString(1).split("="); assertEquals("Property should be set", results.length, 2); assertEquals("Property should be set", expectedValue, results[1]); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java index 15dbe77..949f505 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java @@ -44,6 +44,7 @@ public class SparkHashTableSinkOperator extends TerminalOperator implements Serializable { + private static final int MIN_REPLICATION = 10; private static final long serialVersionUID = 1L; private final String CLASS_NAME = this.getClass().getName(); private final PerfLogger perfLogger = PerfLogger.getPerfLogger(); @@ -122,7 +123,6 @@ protected void flushToFile(MapJoinPersistableTableContainer tableContainer, + "-" + Math.abs(Utilities.randGen.nextInt())); try { // This will guarantee file name uniqueness. - // TODO: can we use the task id, which should be unique if (fs.createNewFile(path)) { break; } @@ -131,10 +131,10 @@ protected void flushToFile(MapJoinPersistableTableContainer tableContainer, } // TODO find out numOfPartitions for the big table int numOfPartitions = replication; - replication = (short)Math.min(10, numOfPartitions); + replication = (short) Math.min(MIN_REPLICATION, numOfPartitions); } - htsOperator.console.printInfo(Utilities.now() + "\tDump the side-table for tag: " + tag + - " with group count: " + tableContainer.size() + " into file: " + path); + htsOperator.console.printInfo(Utilities.now() + "\tDump the side-table for tag: " + tag + + " with group count: " + tableContainer.size() + " into file: " + path); // get the hashtable file and path // get the hashtable file and path OutputStream os = null; @@ -153,8 +153,8 @@ protected void flushToFile(MapJoinPersistableTableContainer tableContainer, } tableContainer.clear(); FileStatus status = fs.getFileStatus(path); - htsOperator.console.printInfo(Utilities.now() + "\tUploaded 1 File to: " + path + - " (" + status.getLen() + " bytes)"); + htsOperator.console.printInfo(Utilities.now() + "\tUploaded 1 File to: " + path + + " (" + status.getLen() + " bytes)"); perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_FLUSH_HASHTABLE + this.getName()); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HashTableLoader.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HashTableLoader.java index ab8010f..129e97b 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HashTableLoader.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HashTableLoader.java @@ -101,7 +101,7 @@ public void load( bigInputPath = null; } else { Set aliases = - ((SparkBucketMapJoinContext)mapJoinCtx).getPosToAliasMap().get(pos); + ((SparkBucketMapJoinContext) mapJoinCtx).getPosToAliasMap().get(pos); String alias = aliases.iterator().next(); // Any one small table input path String smallInputPath = @@ -110,7 +110,7 @@ public void load( } } String fileName = localWork.getBucketFileName(bigInputPath); - Path path = Utilities.generatePath(baseDir, desc.getDumpFilePrefix(), (byte)pos, fileName); + Path path = Utilities.generatePath(baseDir, desc.getDumpFilePrefix(), (byte) pos, fileName); LOG.info("\tLoad back all hashtable files from tmp folder uri:" + path); mapJoinTables[pos] = mapJoinTableSerdes[pos].load(fs, path); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveBaseFunctionResultList.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveBaseFunctionResultList.java index a84660e..9e721ee 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveBaseFunctionResultList.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveBaseFunctionResultList.java @@ -70,13 +70,15 @@ public void collect(HiveKey key, BytesWritable value) throws IOException { /** Process the given record. */ protected abstract void processNextRecord(T inputRecord) throws IOException; - /** Is the current state of the record processor done? */ + /** + * @return true if current state of the record processor is done. + */ protected abstract boolean processingDone(); - /** Close the record processor */ + /** Close the record processor. */ protected abstract void closeRecordProcessor(); - /** Implement Iterator interface */ + /** Implement Iterator interface. */ public class ResultIterator implements Iterator { @Override public boolean hasNext(){ @@ -98,8 +100,7 @@ public boolean hasNext(){ return true; } } catch (IOException ex) { - // TODO: better handling of exception. - throw new RuntimeException("Error while processing input.", ex); + throw new IllegalStateException("Error while processing input.", ex); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveKVResultCache.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveKVResultCache.java index 93dccd6..8ead0cb 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveKVResultCache.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveKVResultCache.java @@ -79,7 +79,7 @@ private static RowContainer initRowContainer(Configuration conf) { container.setSerDe(serDe, oi); container.setTableDesc(tableDesc); - } catch(Exception ex) { + } catch (Exception ex) { throw new RuntimeException("Failed to create RowContainer", ex); } return container; @@ -114,7 +114,7 @@ public synchronized void clear() { } try { container.clearRows(); - } catch(HiveException ex) { + } catch (HiveException ex) { throw new RuntimeException("Failed to clear rows in RowContainer", ex); } cursor = 0; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java index 823ea44..afdacf0 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java @@ -50,7 +50,6 @@ public HiveMapFunction(byte[] jobConfBuffer, SparkReporter sparkReporter) { } HiveMapFunctionResultList result = new HiveMapFunctionResultList(jobConf, it, mapRecordHandler); - //TODO we need to implement a Spark specified Reporter to collect stats, refer to HIVE-7709. mapRecordHandler.init(jobConf, result, sparkReporter); return result; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunctionResultList.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunctionResultList.java index 74650e8..e92e299 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunctionResultList.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunctionResultList.java @@ -19,7 +19,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.mapred.Reporter; import scala.Tuple2; import java.io.IOException; @@ -32,6 +31,7 @@ /** * Instantiate result set Iterable for Map function output. * + * @param conf Hive configuration. * @param inputIterator Input record iterator. * @param handler Initialized {@link SparkMapRecordHandler} instance. */ diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HivePairFlatMapFunction.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HivePairFlatMapFunction.java index 1f75cb7..fcbe887 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HivePairFlatMapFunction.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HivePairFlatMapFunction.java @@ -26,13 +26,13 @@ public abstract class HivePairFlatMapFunction implements PairFlatMapFunction { - private static final NumberFormat taskIdFormat = NumberFormat.getInstance(); - private static final NumberFormat stageIdFormat = NumberFormat.getInstance(); + private static final NumberFormat TASK_ID_FORMAT = NumberFormat.getInstance(); + private static final NumberFormat STAGE_ID_FORMAT = NumberFormat.getInstance(); static { - taskIdFormat.setGroupingUsed(false); - taskIdFormat.setMinimumIntegerDigits(6); - stageIdFormat.setGroupingUsed(false); - stageIdFormat.setMinimumIntegerDigits(4); + TASK_ID_FORMAT.setGroupingUsed(false); + TASK_ID_FORMAT.setMinimumIntegerDigits(6); + STAGE_ID_FORMAT.setGroupingUsed(false); + STAGE_ID_FORMAT.setMinimumIntegerDigits(4); } protected transient JobConf jobConf; @@ -60,7 +60,7 @@ private void setupMRLegacyConfigs() { StringBuilder taskAttemptIdBuilder = new StringBuilder("attempt_"); taskAttemptIdBuilder.append(System.currentTimeMillis()) .append("_") - .append(stageIdFormat.format(TaskContext.get().stageId())) + .append(STAGE_ID_FORMAT.format(TaskContext.get().stageId())) .append("_"); if (isMap()) { @@ -71,7 +71,7 @@ private void setupMRLegacyConfigs() { // Spark task attempt id is increased by Spark context instead of task, which may introduce // unstable qtest output, since non Hive features depends on this, we always set it to 0 here. - taskAttemptIdBuilder.append(taskIdFormat.format(TaskContext.get().partitionId())) + taskAttemptIdBuilder.append(TASK_ID_FORMAT.format(TaskContext.get().partitionId())) .append("_0"); String taskAttemptIdStr = taskAttemptIdBuilder.toString(); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java index 9bd305a..070ea4d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java @@ -20,6 +20,7 @@ import org.apache.hadoop.hive.ql.io.HiveKey; import org.apache.hadoop.io.BytesWritable; + import scala.Tuple2; import java.util.Iterator; @@ -33,6 +34,7 @@ public HiveReduceFunction(byte[] buffer, SparkReporter sparkReporter) { super(buffer, sparkReporter); } + @SuppressWarnings("unchecked") @Override public Iterable> call(Iterator>> it) throws Exception { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunctionResultList.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunctionResultList.java index c153ad8..d4ff37c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunctionResultList.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunctionResultList.java @@ -32,6 +32,7 @@ /** * Instantiate result set Iterable for Reduce function output. * + * @param conf Hive configuration. * @param inputIterator Input record iterator. * @param reducer Initialized {@link org.apache.hadoop.hive.ql.exec.mr.ExecReducer} instance. */ diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java index a456d6c..6b5c133 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java @@ -34,12 +34,15 @@ * @return SparkJobRef could be used to track spark job progress and metrics. * @throws Exception */ - public SparkJobRef execute(DriverContext driverContext, SparkWork sparkWork) throws Exception; + SparkJobRef execute(DriverContext driverContext, SparkWork sparkWork) throws Exception; - public SparkConf getSparkConf(); + /** + * @return spark configuration + */ + SparkConf getSparkConf(); /** - * Get the count of executors + * @return the number of executors */ - public int getExecutorCount() throws Exception; + int getExecutorCount() throws Exception; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java index 78c1da8..2fbf53e 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java @@ -33,12 +33,12 @@ import org.apache.spark.SparkException; public class HiveSparkClientFactory { - protected static transient final Log LOG = LogFactory.getLog(HiveSparkClientFactory.class); + protected static final transient Log LOG = LogFactory.getLog(HiveSparkClientFactory.class); private static final String SPARK_DEFAULT_CONF_FILE = "spark-defaults.conf"; private static final String SPARK_DEFAULT_MASTER = "local"; private static final String SPARK_DEFAULT_APP_NAME = "Hive on Spark"; - + public static HiveSparkClient createHiveSparkClient(HiveConf hiveconf) throws IOException, SparkException { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/KryoSerializer.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/KryoSerializer.java index 6288ff2..8ec2a9e 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/KryoSerializer.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/KryoSerializer.java @@ -27,10 +27,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.mr.ExecMapper; import org.apache.hadoop.mapred.JobConf; -import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java index 5cfdcec..dfb409f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java @@ -56,7 +56,7 @@ private static final long serialVersionUID = 1L; private static final String MR_JAR_PROPERTY = "tmpjars"; - protected static transient final Log LOG = LogFactory + protected static final transient Log LOG = LogFactory .getLog(LocalHiveSparkClient.class); private static final Splitter CSV_SPLITTER = Splitter.on(",").omitEmptyStrings(); @@ -138,7 +138,7 @@ public SparkJobRef execute(DriverContext driverContext, SparkWork sparkWork) thr * At this point single SparkContext is used by more than one thread, so make this * method synchronized. * - * TODO: This method can't remove a jar/resource from SparkContext. Looks like this is an + * This method can't remove a jar/resource from SparkContext. Looks like this is an * issue we have to live with until multiple SparkContexts are supported in a single JVM. */ private synchronized void refreshLocalResources(SparkWork sparkWork, HiveConf conf) { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java index 87b47a6..45c2977 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java @@ -64,10 +64,10 @@ private static final long serialVersionUID = 1L; private static final String MR_JAR_PROPERTY = "tmpjars"; - protected static transient final Log LOG = LogFactory + protected static final transient Log LOG = LogFactory .getLog(RemoteHiveSparkClient.class); - private static transient final Splitter CSV_SPLITTER = Splitter.on(",").omitEmptyStrings(); + private static final transient Splitter CSV_SPLITTER = Splitter.on(",").omitEmptyStrings(); private transient SparkClient remoteClient; private transient SparkConf sparkConf; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java index e65b4ba..066d495 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java @@ -58,16 +58,16 @@ public class SparkMapRecordHandler extends SparkRecordHandler { private static final String PLAN_KEY = "__MAP_PLAN__"; private MapOperator mo; - public static final Log l4j = LogFactory.getLog(SparkMapRecordHandler.class); + public static final Log LOG = LogFactory.getLog(SparkMapRecordHandler.class); private MapredLocalWork localWork = null; private boolean isLogInfoEnabled = false; private ExecMapperContext execContext; - public void init(JobConf job, OutputCollector output, Reporter reporter) { + public void init(JobConf job, OutputCollector output, Reporter reporter) { perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_INIT_OPERATORS); super.init(job, output, reporter); - isLogInfoEnabled = l4j.isInfoEnabled(); + isLogInfoEnabled = LOG.isInfoEnabled(); ObjectCache cache = ObjectCacheFactory.getCache(job); try { @@ -90,7 +90,7 @@ public void init(JobConf job, OutputCollector output, Reporter reporter) { // initialize map operator mo.setChildren(job); - l4j.info(mo.dump(0)); + LOG.info(mo.dump(0)); // initialize map local work localWork = mrwork.getMapRedLocalWork(); execContext.setLocalWork(localWork); @@ -111,11 +111,11 @@ public void init(JobConf job, OutputCollector output, Reporter reporter) { //The following code is for mapjoin //initialize all the dummy ops - l4j.info("Initializing dummy operator"); + LOG.info("Initializing dummy operator"); List> dummyOps = localWork.getDummyParentOp(); - for (Operator dummyOp : dummyOps){ + for (Operator dummyOp : dummyOps) { dummyOp.setExecContext(execContext); - dummyOp.initialize(jc,null); + dummyOp.initialize(jc, null); } } catch (Throwable e) { abort = true; @@ -148,14 +148,14 @@ public void processRow(Object key, Object value) throws IOException { // Don't create a new object if we are already out of memory throw (OutOfMemoryError) e; } else { - l4j.fatal(StringUtils.stringifyException(e)); + LOG.fatal(StringUtils.stringifyException(e)); throw new RuntimeException(e); } } } @Override - public void processRow(Object key, Iterator values) throws IOException { + public void processRow(Object key, Iterator values) throws IOException { throw new UnsupportedOperationException("Do not support this method in SparkMapRecordHandler."); } @@ -163,7 +163,7 @@ public void processRow(Object key, Iterator values) throws IOException { public void close() { // No row was processed if (oc == null) { - l4j.trace("Close called. no row processed by map."); + LOG.trace("Close called. no row processed by map."); } // check if there are IOExceptions @@ -177,10 +177,10 @@ public void close() { mo.close(abort); //for close the local work - if(localWork != null){ + if (localWork != null) { List> dummyOps = localWork.getDummyParentOp(); - for (Operator dummyOp : dummyOps){ + for (Operator dummyOp : dummyOps) { dummyOp.close(abort); } } @@ -195,7 +195,7 @@ public void close() { } catch (Exception e) { if (!abort) { // signal new failure to map-reduce - l4j.error("Hit error while closing operators - failing tree"); + LOG.error("Hit error while closing operators - failing tree"); throw new RuntimeException("Hive Runtime Error while closing operators", e); } } finally { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMergeFileRecordHandler.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMergeFileRecordHandler.java index 27f6464..9112bf2 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMergeFileRecordHandler.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMergeFileRecordHandler.java @@ -18,24 +18,27 @@ package org.apache.hadoop.hive.ql.exec.spark; +import java.io.IOException; +import java.util.Iterator; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.ql.exec.*; -import org.apache.hadoop.hive.ql.exec.mr.ExecMapperContext; +import org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator; +import org.apache.hadoop.hive.ql.exec.ObjectCache; +import org.apache.hadoop.hive.ql.exec.ObjectCacheFactory; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.io.merge.MergeFileWork; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.FileMergeDesc; import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; -import java.io.IOException; -import java.util.Iterator; - /** - * Copied from MergeFileMapper + * Copied from MergeFileMapper. * * As MergeFileMapper is very similar to ExecMapper, this class is * very similar to SparkMapRecordHandler @@ -43,13 +46,14 @@ public class SparkMergeFileRecordHandler extends SparkRecordHandler { private static final String PLAN_KEY = "__MAP_PLAN__"; - private static final Log l4j = LogFactory.getLog(SparkMergeFileRecordHandler.class); + private static final Log LOG = LogFactory.getLog(SparkMergeFileRecordHandler.class); private Operator op; - private AbstractFileMergeOperator mergeOp; + private AbstractFileMergeOperator mergeOp; private Object[] row; + @SuppressWarnings("unchecked") @Override - public void init(JobConf job, OutputCollector output, Reporter reporter) { + public void init(JobConf job, OutputCollector output, Reporter reporter) { super.init(job, output, reporter); ObjectCache cache = ObjectCacheFactory.getCache(job); @@ -70,22 +74,22 @@ public void init(JobConf job, OutputCollector output, Reporter reporter) { String alias = mergeFileWork.getAliasToWork().keySet().iterator().next(); op = mergeFileWork.getAliasToWork().get(alias); if (op instanceof AbstractFileMergeOperator) { - mergeOp = (AbstractFileMergeOperator) op; + mergeOp = (AbstractFileMergeOperator) op; mergeOp.initializeOp(jc); row = new Object[2]; abort = false; } else { abort = true; throw new RuntimeException( - "Merge file work's top operator should be an" + - " instance of AbstractFileMergeOperator"); + "Merge file work's top operator should be an" + + " instance of AbstractFileMergeOperator"); } } else { abort = true; throw new RuntimeException("Map work should be a merge file work."); } - l4j.info(mergeOp.dump(0)); + LOG.info(mergeOp.dump(0)); } catch (HiveException e) { abort = true; throw new RuntimeException(e); @@ -105,14 +109,14 @@ public void processRow(Object key, Object value) throws IOException { } @Override - public void processRow(Object key, Iterator values) throws IOException { + public void processRow(Object key, Iterator values) throws IOException { throw new UnsupportedOperationException("Do not support this method in " + this.getClass().getSimpleName()); } @Override public void close() { - l4j.info("Closing Merge Operator " + mergeOp.getName()); + LOG.info("Closing Merge Operator " + mergeOp.getName()); try { mergeOp.closeOp(abort); } catch (HiveException e) { @@ -121,7 +125,7 @@ public void close() { } @Override - public boolean getDone() { + public boolean getDone() { return mergeOp.getDone(); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlan.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlan.java index fb08950..b45494d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlan.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlan.java @@ -33,8 +33,9 @@ import com.google.common.base.Preconditions; +@SuppressWarnings("rawtypes") public class SparkPlan { - private final String CLASS_NAME = SparkPlan.class.getName(); + private static final String CLASS_NAME = SparkPlan.class.getName(); private final PerfLogger perfLogger = PerfLogger.getPerfLogger(); private final Set rootTrans = new HashSet(); @@ -43,7 +44,8 @@ private final Map> invertedTransGraph = new HashMap>(); private final Set cachedRDDIds = new HashSet(); - public JavaPairRDD generateGraph() throws IllegalStateException { + @SuppressWarnings("unchecked") + public JavaPairRDD generateGraph() { perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_BUILD_RDD_GRAPH); Map> tranToOutputRDDMap = new HashMap>(); @@ -98,7 +100,7 @@ public void addCachedRDDId(int rddId) { } /** - * This method returns a topologically sorted list of SparkTran + * This method returns a topologically sorted list of SparkTran. */ private List getAllTrans() { List result = new LinkedList(); @@ -135,7 +137,7 @@ private void visit(SparkTran child, Set seen, List result) * @param parent * @param child */ - public void connect(SparkTran parent, SparkTran child) throws IllegalStateException { + public void connect(SparkTran parent, SparkTran child) { if (getChildren(parent).contains(child)) { throw new IllegalStateException("Connection already exists"); } @@ -151,7 +153,7 @@ public void connect(SparkTran parent, SparkTran child) throws IllegalStateExcept invertedTransGraph.get(child).add(parent); } - public List getParents(SparkTran tran) throws IllegalStateException { + public List getParents(SparkTran tran) { if (!invertedTransGraph.containsKey(tran)) { return new ArrayList(); } @@ -159,7 +161,7 @@ public void connect(SparkTran parent, SparkTran child) throws IllegalStateExcept return invertedTransGraph.get(tran); } - public List getChildren(SparkTran tran) throws IllegalStateException { + public List getChildren(SparkTran tran) { if (!transGraph.containsKey(tran)) { return new ArrayList(); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java index ae73f8a..1016a7c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java @@ -23,6 +23,7 @@ import java.util.Map; import com.google.common.base.Preconditions; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.Path; @@ -53,9 +54,9 @@ import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaSparkContext; - +@SuppressWarnings("rawtypes") public class SparkPlanGenerator { - private final String CLASS_NAME = SparkPlanGenerator.class.getName(); + private static final String CLASS_NAME = SparkPlanGenerator.class.getName(); private final PerfLogger perfLogger = PerfLogger.getPerfLogger(); private static final Log LOG = LogFactory.getLog(SparkPlanGenerator.class); @@ -131,8 +132,8 @@ private SparkTran generateParentTran(SparkPlan sparkPlan, SparkWork sparkWork, sparkPlan.connect(workToTranMap.get(parentWork), result); } } else { - throw new IllegalStateException("AssertionError: expected either MapWork or ReduceWork, " + - "but found " + work.getClass().getName()); + throw new IllegalStateException("AssertionError: expected either MapWork or ReduceWork, " + + "but found " + work.getClass().getName()); } if (cloneToWork.containsKey(work)) { @@ -142,7 +143,7 @@ private SparkTran generateParentTran(SparkPlan sparkPlan, SparkWork sparkWork, return result; } - private Class getInputFormat(JobConf jobConf, MapWork mWork) throws HiveException { + private Class getInputFormat(JobConf jobConf, MapWork mWork) throws HiveException { // MergeFileWork is sub-class of MapWork, we don't need to distinguish here if (mWork.getInputformat() != null) { HiveConf.setVar(jobConf, HiveConf.ConfVars.HIVEINPUTFORMAT, @@ -168,6 +169,7 @@ private Class getInputFormat(JobConf jobConf, MapWork mWork) throws HiveExceptio return inputFormatClass; } + @SuppressWarnings("unchecked") private MapInput generateMapInput(SparkPlan sparkPlan, MapWork mapWork) throws Exception { JobConf jobConf = cloneJobConf(mapWork); @@ -209,11 +211,12 @@ private SparkTran generate(BaseWork work) throws Exception { reduceTran.setReduceFunction(reduceFunc); return reduceTran; } else { - throw new IllegalStateException("AssertionError: expected either MapWork or ReduceWork, " + - "but found " + work.getClass().getName()); + throw new IllegalStateException("AssertionError: expected either MapWork or ReduceWork, " + + "but found " + work.getClass().getName()); } } + @SuppressWarnings({ "unchecked" }) private JobConf cloneJobConf(BaseWork work) throws Exception { if (workToJobConf.containsKey(work)) { return workToJobConf.get(work); @@ -225,9 +228,8 @@ private JobConf cloneJobConf(BaseWork work) throws Exception { cloned.setPartitionerClass((Class) (Class.forName(HiveConf.getVar(cloned, HiveConf.ConfVars.HIVEPARTITIONER)))); } catch (ClassNotFoundException e) { - String msg = "Could not find partitioner class: " + e.getMessage() + - " which is specified by: " + - HiveConf.ConfVars.HIVEPARTITIONER.varname; + String msg = "Could not find partitioner class: " + e.getMessage() + + " which is specified by: " + HiveConf.ConfVars.HIVEPARTITIONER.varname; throw new IllegalArgumentException(msg, e); } if (work instanceof MapWork) { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java index 08eb4fb..c29495e 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java @@ -34,9 +34,9 @@ import java.util.Iterator; public abstract class SparkRecordHandler { - protected final String CLASS_NAME = this.getClass().getName(); + protected static final String CLASS_NAME = SparkRecordHandler.class.getName(); protected final PerfLogger perfLogger = PerfLogger.getPerfLogger(); - private final Log LOG = LogFactory.getLog(this.getClass()); + private static final Log LOG = LogFactory.getLog(SparkRecordHandler.class); // used to log memory usage periodically protected final MemoryMXBean memoryMXBean = ManagementFactory.getMemoryMXBean(); @@ -48,14 +48,13 @@ private long rowNumber = 0; private long nextLogThreshold = 1; - public void init(JobConf job, OutputCollector output, Reporter reporter) { + public void init(JobConf job, OutputCollector output, Reporter reporter) { jc = job; MapredContext.init(false, new JobConf(jc)); MapredContext.get().setReporter(reporter); oc = output; rp = reporter; -// MapredContext.get().setReporter(reporter); LOG.info("maximum memory = " + memoryMXBean.getHeapMemoryUsage().getMax()); @@ -78,7 +77,7 @@ public void init(JobConf job, OutputCollector output, Reporter reporter) { /** * Process row with key and value collection. */ - public abstract void processRow(Object key, Iterator values) throws IOException; + public abstract void processRow(Object key, Iterator values) throws IOException; /** * Log processed row number and used memory info. @@ -86,9 +85,9 @@ public void init(JobConf job, OutputCollector output, Reporter reporter) { protected void logMemoryInfo() { rowNumber++; if (rowNumber == nextLogThreshold) { - long used_memory = memoryMXBean.getHeapMemoryUsage().getUsed(); + long usedMemory = memoryMXBean.getHeapMemoryUsage().getUsed(); LOG.info("processing " + rowNumber - + " rows: used memory = " + used_memory); + + " rows: used memory = " + usedMemory); nextLogThreshold = getNextLogThreshold(rowNumber); } } @@ -97,12 +96,12 @@ protected void logMemoryInfo() { public abstract boolean getDone(); /** - * Log information to be logged at the end + * Log information to be logged at the end. */ protected void logCloseInfo() { - long used_memory = memoryMXBean.getHeapMemoryUsage().getUsed(); + long usedMemory = memoryMXBean.getHeapMemoryUsage().getUsed(); LOG.info("processed " + rowNumber + " rows: used memory = " - + used_memory); + + usedMemory); } private long getNextLogThreshold(long currentThreshold) { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java index 0a32fff..dd9bfee 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java @@ -69,7 +69,7 @@ * - Catch and handle errors during execution of the operators. * */ -public class SparkReduceRecordHandler extends SparkRecordHandler{ +public class SparkReduceRecordHandler extends SparkRecordHandler { private static final Log LOG = LogFactory.getLog(SparkReduceRecordHandler.class); private static final String PLAN_KEY = "__REDUCE_PLAN__"; @@ -98,13 +98,14 @@ private VectorizedRowBatch[] batches; // number of columns pertaining to keys in a vectorized row batch private int keysColumnOffset; - private final int BATCH_SIZE = VectorizedRowBatch.DEFAULT_SIZE; + private static final int BATCH_SIZE = VectorizedRowBatch.DEFAULT_SIZE; private StructObjectInspector keyStructInspector; private StructObjectInspector[] valueStructInspectors; /* this is only used in the error code path */ private List[] valueStringWriters; private MapredLocalWork localWork = null; + @SuppressWarnings("unchecked") public void init(JobConf job, OutputCollector output, Reporter reporter) { perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_INIT_OPERATORS); super.init(job, output, reporter); @@ -240,7 +241,7 @@ public void processRow(Object key, Object value) throws IOException { } @Override - public void processRow(Object key, Iterator values) throws IOException { + public void processRow(Object key, Iterator values) throws IOException { if (reducer.getDone()) { return; } @@ -305,7 +306,7 @@ public void processRow(Object key, Iterator values) throws IOException { * @param values * @return true if it is not done and can take more inputs */ - private boolean processKeyValues(Iterator values, byte tag) throws HiveException { + private boolean processKeyValues(Iterator values, byte tag) throws HiveException { while (values.hasNext()) { BytesWritable valueWritable = (BytesWritable) values.next(); try { @@ -332,8 +333,8 @@ private boolean processKeyValues(Iterator values, byte tag) throws HiveException try { rowString = SerDeUtils.getJSONString(row, rowObjectInspector[tag]); } catch (Exception e2) { - rowString = "[Error getting row data with exception " + - StringUtils.stringifyException(e2) + " ]"; + rowString = "[Error getting row data with exception " + + StringUtils.stringifyException(e2) + " ]"; } throw new HiveException("Hive Runtime Error while processing row (tag=" + tag + ") " + rowString, e); @@ -346,7 +347,7 @@ private boolean processKeyValues(Iterator values, byte tag) throws HiveException * @param values * @return true if it is not done and can take more inputs */ - private boolean processVectors(Iterator values, byte tag) throws HiveException { + private boolean processVectors(Iterator values, byte tag) throws HiveException { VectorizedRowBatch batch = batches[tag]; batch.reset(); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReporter.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReporter.java index fdf43f1..0b3c970 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReporter.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReporter.java @@ -67,7 +67,7 @@ public void incrCounter(String group, String counter, long amount) { } @Override - public InputSplit getInputSplit() throws UnsupportedOperationException { + public InputSplit getInputSplit() { throw new UnsupportedOperationException("do not support this method now."); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java index db8bcf6..ac2aa25 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java @@ -28,7 +28,6 @@ import java.util.List; import java.util.Map; -import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.Warehouse; @@ -47,14 +46,13 @@ import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatistic; import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticGroup; import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatistics; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hive.spark.counter.SparkCounters; import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession; import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManager; import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobMonitor; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobStatus; +import org.apache.hadoop.hive.ql.log.PerfLogger; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; @@ -70,23 +68,20 @@ import org.apache.hadoop.hive.ql.plan.UnionWork; import org.apache.hadoop.hive.ql.plan.api.StageType; import org.apache.hadoop.hive.ql.stats.StatsFactory; -import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.util.StringUtils; +import org.apache.hive.spark.counter.SparkCounters; import com.google.common.collect.Lists; public class SparkTask extends Task { - private final String CLASS_NAME = SparkTask.class.getName(); + private static final String CLASS_NAME = SparkTask.class.getName(); private final PerfLogger perfLogger = PerfLogger.getPerfLogger(); private static final long serialVersionUID = 1L; - private transient JobConf job; - private transient ContentSummary inputSummary; private SparkCounters sparkCounters; @Override public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext) { super.initialize(conf, queryPlan, driverContext); - job = new JobConf(conf, SparkTask.class); } @Override @@ -133,7 +128,7 @@ public int execute(DriverContext driverContext) { rc = close(rc); try { sparkSessionManager.returnSession(sparkSession); - } catch(HiveException ex) { + } catch (HiveException ex) { LOG.error("Failed to return the session to SessionManager", ex); } } @@ -155,7 +150,7 @@ private void logSparkStatistic(SparkStatistics sparkStatistic) { } /** - * close will move the temp files into the right place for the fetch + * Close will move the temp files into the right place for the fetch * task. If the job has failed it will clean up the files. */ private int close(int rc) { @@ -211,7 +206,7 @@ public String getName() { } } if (candidate) { - result.add((MapWork)w); + result.add((MapWork) w); } } } @@ -316,11 +311,11 @@ private static StatsTask getStatsTaskInChildTasks(Task r } for (Task task : childTasks) { if (task instanceof StatsTask) { - return (StatsTask)task; + return (StatsTask) task; } else { Task childTask = getStatsTaskInChildTasks(task); if (childTask instanceof StatsTask) { - return (StatsTask)childTask; + return (StatsTask) childTask; } else { continue; } @@ -383,7 +378,7 @@ private static StatsTask getStatsTaskInChildTasks(Task r } SparkWork sparkWork = this.getWork(); for (BaseWork work : sparkWork.getAllWork()) { - for (Operator operator : work.getAllOperators()) { + for (Operator operator : work.getAllOperators()) { if (operator instanceof FileSinkOperator) { for (FileSinkOperator.Counter counter : FileSinkOperator.Counter.values()) { hiveCounters.add(counter.toString()); @@ -392,11 +387,11 @@ private static StatsTask getStatsTaskInChildTasks(Task r for (ReduceSinkOperator.Counter counter : ReduceSinkOperator.Counter.values()) { hiveCounters.add(counter.toString()); } - }else if (operator instanceof ScriptOperator) { + } else if (operator instanceof ScriptOperator) { for (ScriptOperator.Counter counter : ScriptOperator.Counter.values()) { hiveCounters.add(counter.toString()); } - }else if (operator instanceof JoinOperator) { + } else if (operator instanceof JoinOperator) { for (JoinOperator.SkewkeyTableCounter counter : JoinOperator.SkewkeyTableCounter.values()) { hiveCounters.add(counter.toString()); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTran.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTran.java index c413952..4daa61e 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTran.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTran.java @@ -21,6 +21,7 @@ import org.apache.hadoop.io.WritableComparable; import org.apache.spark.api.java.JavaPairRDD; +@SuppressWarnings("rawtypes") public interface SparkTran { JavaPairRDD transform( JavaPairRDD input); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java index b80122b..20f542d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java @@ -32,7 +32,7 @@ import org.apache.hadoop.io.BytesWritable; /** - * Contains utilities methods used as part of Spark tasks + * Contains utilities methods used as part of Spark tasks. */ public class SparkUtilities { @@ -76,7 +76,7 @@ public static SparkSession getSparkSession(HiveConf conf, SparkSession sparkSession = SessionState.get().getSparkSession(); // Spark configurations are updated close the existing session - if(conf.getSparkConfigUpdated()){ + if (conf.getSparkConfigUpdated()) { sparkSessionManager.closeSession(sparkSession); sparkSession = null; conf.setSparkConfigUpdated(false); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java index 461f359..28748fb 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java @@ -28,15 +28,17 @@ public interface SparkSession { /** * Initializes a Spark session for DAG execution. + * @param conf Hive configuration. */ - public void open(HiveConf conf) throws HiveException; + void open(HiveConf conf) throws HiveException; /** - * Submit given sparkWork to SparkClient + * Submit given sparkWork to SparkClient. * @param driverContext * @param sparkWork + * @return SparkJobRef */ - public SparkJobRef submit(DriverContext driverContext, SparkWork sparkWork) throws Exception; + SparkJobRef submit(DriverContext driverContext, SparkWork sparkWork) throws Exception; /** * Get Spark shuffle memory per task, and total number of cores. This @@ -45,25 +47,25 @@ * @return a tuple, the first element is the shuffle memory per task in bytes, * the second element is the number of total cores usable by the client */ - public Tuple2 getMemoryAndCores() throws Exception; + Tuple2 getMemoryAndCores() throws Exception; /** - * Is the session open and ready to submit jobs? + * @return true if the session is open and ready to submit jobs. */ - public boolean isOpen(); + boolean isOpen(); /** - * Return configuration. + * @return configuration. */ - public HiveConf getConf(); + HiveConf getConf(); /** - * Return session id. + * @return session id. */ - public String getSessionId(); + String getSessionId(); /** - * Close session and release resources + * Close session and release resources. */ - public void close(); + void close(); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManager.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManager.java index b7ec5f3..4c47739 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManager.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManager.java @@ -30,7 +30,7 @@ * * @param hiveConf */ - public void setup(HiveConf hiveConf) throws HiveException; + void setup(HiveConf hiveConf) throws HiveException; /** * Get a valid SparkSession. First try to check if existing session is reusable @@ -40,9 +40,9 @@ * @param existingSession Existing session (can be null) * @param conf * @param doOpen Should the session be opened before returning? - * @return + * @return SparkSession */ - public SparkSession getSession(SparkSession existingSession, HiveConf conf, + SparkSession getSession(SparkSession existingSession, HiveConf conf, boolean doOpen) throws HiveException; /** @@ -50,16 +50,16 @@ public SparkSession getSession(SparkSession existingSession, HiveConf conf, * still holds references to session and may want to reuse it in future. * When client wants to reuse the session, it should pass the it getSession method. */ - public void returnSession(SparkSession sparkSession) throws HiveException; + void returnSession(SparkSession sparkSession) throws HiveException; /** * Close the given session and return it to pool. This is used when the client * no longer needs a SparkSession. */ - public void closeSession(SparkSession sparkSession) throws HiveException; + void closeSession(SparkSession sparkSession) throws HiveException; /** * Shutdown the session manager. Also closing up SparkSessions in pool. */ - public void shutdown(); + void shutdown(); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java index 2e80383..45fc05d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java @@ -64,7 +64,7 @@ public void run() { }); } - public synchronized static SparkSessionManagerImpl getInstance() + public static synchronized SparkSessionManagerImpl getInstance() throws HiveException { if (instance == null) { instance = new SparkSessionManagerImpl(); @@ -142,12 +142,12 @@ private boolean canReuseSession(SparkSession existingSession, HiveConf conf) thr UserGroupInformation newUgi = Utils.getUGI(); String newUserName = newUgi.getShortUserName(); - // TODOD this we need to store the session username somewhere else as getUGIForConf never used the conf + // TODO this we need to store the session username somewhere else as getUGIForConf never used the conf UserGroupInformation ugiInSession = Utils.getUGI(); String userNameInSession = ugiInSession.getShortUserName(); return newUserName.equals(userNameInSession); - } catch(Exception ex) { + } catch (Exception ex) { throw new HiveException("Failed to get user info from HiveConf.", ex); } } @@ -175,7 +175,7 @@ public void closeSession(SparkSession sparkSession) throws HiveException { public void shutdown() { LOG.info("Closing the session manager."); if (createdSessions != null) { - synchronized(createdSessions) { + synchronized (createdSessions) { Iterator it = createdSessions.iterator(); while (it.hasNext()) { SparkSession session = it.next(); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java index ff65418..999011b 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java @@ -74,8 +74,8 @@ public int startMonitor() { if (LOG.isDebugEnabled()) { console.printInfo("state = " + state); } - if (state != null && state != JobExecutionStatus.UNKNOWN && - (state != lastState || state == JobExecutionStatus.RUNNING)) { + if (state != null && state != JobExecutionStatus.UNKNOWN + && (state != lastState || state == JobExecutionStatus.RUNNING)) { lastState = state; Map progressMap = sparkJobStatus.getSparkStageProgress(); @@ -84,19 +84,19 @@ public int startMonitor() { if (!running) { perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); // print job stages. - console.printInfo("\nQuery Hive on Spark job[" + - sparkJobStatus.getJobId() + "] stages:"); + console.printInfo("\nQuery Hive on Spark job[" + + sparkJobStatus.getJobId() + "] stages:"); for (int stageId : sparkJobStatus.getStageIds()) { console.printInfo(Integer.toString(stageId)); } - console.printInfo("\nStatus: Running (Hive on Spark job[" + - sparkJobStatus.getJobId() + "])"); + console.printInfo("\nStatus: Running (Hive on Spark job[" + + sparkJobStatus.getJobId() + "])"); startTime = System.currentTimeMillis(); running = true; - console.printInfo("Job Progress Format\nCurrentTime StageId_StageAttemptId: " + - "SucceededTasksCount(+RunningTasksCount-FailedTasksCount)/TotalTasksCount [StageCost]"); + console.printInfo("Job Progress Format\nCurrentTime StageId_StageAttemptId: " + + "SucceededTasksCount(+RunningTasksCount-FailedTasksCount)/TotalTasksCount [StageCost]"); } @@ -110,8 +110,8 @@ public int startMonitor() { console.printInfo("Status: Finished successfully within a check interval."); } else { double duration = (System.currentTimeMillis() - startTime) / 1000.0; - console.printInfo("Status: Finished successfully in " + - String.format("%.2f seconds", duration)); + console.printInfo("Status: Finished successfully in " + + String.format("%.2f seconds", duration)); } running = false; done = true; @@ -122,6 +122,12 @@ public int startMonitor() { done = true; rc = 2; break; + case UNKNOWN: + console.printError("Status: Unknown"); + running = false; + done = true; + rc = 2; + break; } } if (!done) { @@ -231,11 +237,7 @@ private boolean isSameAsPreviousProgress( } if (progressMap.isEmpty()) { - if (lastProgressMap.isEmpty()) { - return true; - } else { - return false; - } + return lastProgressMap.isEmpty(); } else { if (lastProgressMap.isEmpty()) { return false; @@ -244,8 +246,8 @@ private boolean isSameAsPreviousProgress( return false; } for (String key : progressMap.keySet()) { - if (!lastProgressMap.containsKey(key) || - !progressMap.get(key).equals(lastProgressMap.get(key))) { + if (!lastProgressMap.containsKey(key) + || !progressMap.get(key).equals(lastProgressMap.get(key))) { return false; } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java index d16d1b4..cb84556 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java @@ -23,7 +23,7 @@ private SparkJobStatus sparkJobStatus; - public SparkJobRef() {} + public SparkJobRef() { } public SparkJobRef(String jobId) { this.jobId = jobId; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java index 91ecefa..cdbc59c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java @@ -28,17 +28,17 @@ */ public interface SparkJobStatus { - public int getJobId(); + int getJobId(); - public JobExecutionStatus getState(); + JobExecutionStatus getState(); - public int[] getStageIds(); + int[] getStageIds(); - public Map getSparkStageProgress(); + Map getSparkStageProgress(); - public SparkCounters getCounter(); + SparkCounters getCounter(); - public SparkStatistics getSparkStatistics(); + SparkStatistics getSparkStatistics(); - public void cleanup(); + void cleanup(); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkStageProgress.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkStageProgress.java index cfec354..65d67ac 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkStageProgress.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkStageProgress.java @@ -23,10 +23,6 @@ private int succeededTaskCount; private int runningTaskCount; private int failedTaskCount; - // TODO: remove the following two metrics as they're not available in current spark API, - // we can add them back once spark provides it -// private int killedTaskCount; -// private long cumulativeTime; public SparkStageProgress( int totalTaskCount, diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java index 931ec7f..8243d4d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java @@ -21,13 +21,9 @@ import java.util.List; import java.util.Map; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobState; import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.scheduler.JobSucceeded; import org.apache.spark.scheduler.SparkListener; import org.apache.spark.scheduler.SparkListenerApplicationEnd; import org.apache.spark.scheduler.SparkListenerApplicationStart; @@ -44,9 +40,12 @@ import org.apache.spark.scheduler.SparkListenerTaskStart; import org.apache.spark.scheduler.SparkListenerUnpersistRDD; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + public class JobMetricsListener implements SparkListener { - private final static Log LOG = LogFactory.getLog(JobMetricsListener.class); + private static final Log LOG = LogFactory.getLog(JobMetricsListener.class); private final Map jobIdToStageId = Maps.newHashMap(); private final Map stageIdToJobId = Maps.newHashMap(); @@ -100,7 +99,7 @@ public synchronized void onJobStart(SparkListenerJobStart jobStart) { int jobId = jobStart.jobId(); int size = jobStart.stageIds().size(); int[] intStageIds = new int[size]; - for(int i=0; i< size; i++) { + for (int i = 0; i < size; i++) { Integer stageId = (Integer) jobStart.stageIds().apply(i); intStageIds[i] = stageId; stageIdToJobId.put(stageId, jobId); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobStatus.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobStatus.java index f66ed42..86360e5 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobStatus.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobStatus.java @@ -99,8 +99,8 @@ public JobExecutionStatus getState() { int totalTaskCount = sparkStageInfo.numTasks(); SparkStageProgress sparkStageProgress = new SparkStageProgress( totalTaskCount, completedTaskCount, runningTaskCount, failedTaskCount); - stageProgresses.put(String.valueOf(sparkStageInfo.stageId()) + "_" + - sparkStageInfo.currentAttemptId(), sparkStageProgress); + stageProgresses.put(String.valueOf(sparkStageInfo.stageId()) + "_" + + sparkStageInfo.currentAttemptId(), sparkStageProgress); } } return stageProgresses; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java index 0d03e9d..f1ac9c9 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java @@ -94,8 +94,8 @@ public JobExecutionStatus getState() { int totalTaskCount = sparkStageInfo.numTasks(); SparkStageProgress sparkStageProgress = new SparkStageProgress( totalTaskCount, completedTaskCount, runningTaskCount, failedTaskCount); - stageProgresses.put(String.valueOf(sparkStageInfo.stageId()) + "_" + - sparkStageInfo.currentAttemptId(), sparkStageProgress); + stageProgresses.put(String.valueOf(sparkStageInfo.stageId()) + "_" + + sparkStageInfo.currentAttemptId(), sparkStageProgress); } } return stageProgresses; @@ -132,8 +132,8 @@ public void cleanup() { } private SparkJobInfo getSparkJobInfo() { - Integer sparkJobId = jobHandle.getSparkJobIds().size() == 1 ? - jobHandle.getSparkJobIds().get(0) : null; + Integer sparkJobId = jobHandle.getSparkJobIds().size() == 1 + ? jobHandle.getSparkJobIds().get(0) : null; if (sparkJobId == null) { long duration = TimeUnit.MILLISECONDS.convert( System.nanoTime() - startTime, TimeUnit.NANOSECONDS); diff --git ql/src/java/org/apache/hadoop/hive/ql/lib/TypeRule.java ql/src/java/org/apache/hadoop/hive/ql/lib/TypeRule.java index b55805c..090d163 100644 --- ql/src/java/org/apache/hadoop/hive/ql/lib/TypeRule.java +++ ql/src/java/org/apache/hadoop/hive/ql/lib/TypeRule.java @@ -18,17 +18,16 @@ package org.apache.hadoop.hive.ql.lib; -import org.apache.hadoop.hive.ql.parse.SemanticException; - import java.util.Stack; -import java.util.regex.Matcher; + +import org.apache.hadoop.hive.ql.parse.SemanticException; /** * Rule that matches a particular type of node. */ public class TypeRule implements Rule { - private Class nodeClass; + private Class nodeClass; public TypeRule(Class nodeClass) { this.nodeClass = nodeClass; diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java index 9f54916..ce010e3 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.optimizer.physical; import com.google.common.base.Preconditions; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.Path; @@ -79,6 +80,7 @@ private GenSparkSkewJoinProcessor() { // prevent instantiation } + @SuppressWarnings("unchecked") public static void processSkewJoin(JoinOperator joinOp, Task currTask, ReduceWork reduceWork, ParseContext parseCtx) throws SemanticException { @@ -138,7 +140,7 @@ public static void processSkewJoin(JoinOperator joinOp, Task newJoinValueTblDesc = new ArrayList(); - for (Byte tag : tags) { + for (int i = 0; i < tags.length; i++) { newJoinValueTblDesc.add(null); } @@ -231,14 +233,14 @@ public static void processSkewJoin(JoinOperator joinOp, Task ts = GenMapRedUtils.createTemporaryTableScanOperator(rowSchemaList.get((byte) k)); - ((TableScanOperator)ts).setTableDesc(tableDescList.get((byte)k)); + ((TableScanOperator) ts).setTableDesc(tableDescList.get((byte) k)); parentOps[k] = ts; } // create the MapJoinOperator - String dumpFilePrefix = "mapfile"+ PlanUtils.getCountForMapJoinDumpFilePrefix(); + String dumpFilePrefix = "mapfile" + PlanUtils.getCountForMapJoinDumpFilePrefix(); MapJoinDesc mapJoinDescriptor = new MapJoinDesc(newJoinKeys, keyTblDesc, - newJoinValues, newJoinValueTblDesc, newJoinValueTblDesc,joinDescriptor + newJoinValues, newJoinValueTblDesc, newJoinValueTblDesc, joinDescriptor .getOutputColumnNames(), i, joinDescriptor.getConds(), joinDescriptor.getFilters(), joinDescriptor.getNoOuterJoin(), dumpFilePrefix); mapJoinDescriptor.setTagOrder(tags); @@ -307,7 +309,7 @@ public static void processSkewJoin(JoinOperator joinOp, Task currTask, HiveConf hiveConf) { List> children = currTask.getChildTasks(); - return GenMRSkewJoinProcessor.skewJoinEnabled(hiveConf, joinOp) && - (children == null || children.size() <= 1); + return GenMRSkewJoinProcessor.skewJoinEnabled(hiveConf, joinOp) + && (children == null || children.size() <= 1); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkCrossProductCheck.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkCrossProductCheck.java index f6ca3d0..92d2191 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkCrossProductCheck.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkCrossProductCheck.java @@ -58,6 +58,7 @@ @Override public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) throws SemanticException { + @SuppressWarnings("unchecked") Task currTask = (Task) nd; if (currTask instanceof SparkTask) { SparkWork sparkWork = ((SparkTask) currTask).getWork(); diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java index 18d1545..05748a1 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java @@ -240,8 +240,6 @@ private void generateLocalWork(SparkTask originalTask) { } } } - - // TODO: enable non-staged mapjoin } } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java index 613b5bc..64c79df 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java @@ -96,8 +96,8 @@ public Object process(Node nd, Stack stack, long numberOfBytes = 0; // we need to add up all the estimates from the siblings of this reduce sink - for (Operator sibling: - sink.getChildOperators().get(0).getParentOperators()) { + for (Operator sibling + : sink.getChildOperators().get(0).getParentOperators()) { if (sibling.getStatistics() != null) { numberOfBytes += sibling.getStatistics().getDataSize(); if (LOG.isDebugEnabled()) { @@ -139,8 +139,8 @@ public Object process(Node nd, Stack stack, if (numReducers < cores) { numReducers = cores; } - LOG.info("Set parallelism parameters: cores = " + cores + ", numReducers = " + numReducers + - ", bytesPerReducer = " + bytesPerReducer + ", numberOfBytes = " + numberOfBytes); + LOG.info("Set parallelism parameters: cores = " + cores + ", numReducers = " + numReducers + + ", bytesPerReducer = " + bytesPerReducer + ", numberOfBytes = " + numberOfBytes); LOG.info("Set parallelism for reduce sink " + sink + " to: " + numReducers + " (calculated)"); desc.setNumReducers(numReducers); } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinHintOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinHintOptimizer.java index 252ab26..53b7b07 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinHintOptimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinHintOptimizer.java @@ -18,8 +18,9 @@ package org.apache.hadoop.hive.ql.optimizer.spark; +import java.util.Stack; + import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.lib.NodeProcessor; import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; @@ -30,15 +31,13 @@ import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext; -import java.util.Stack; - /** * This processes joins in which user specified a hint to identify the small-table. - * Currently it takes a mapjoin already converted from hints, and converts it further to BucketMapJoin or SMBMapJoin - * using same small-table identification. + * Currently it takes a mapjoin already converted from hints, and converts it further + * to BucketMapJoin or SMBMapJoin using same small-table identification. * - * The idea is eventually to process even hinted Mapjoin hints here, but due to code complexity in refactoring, that is still - * in Optimizer. + * The idea is eventually to process even hinted Mapjoin hints here, + * but due to code complexity in refactoring, that is still in Optimizer. */ public class SparkJoinHintOptimizer implements NodeProcessor { @@ -51,13 +50,14 @@ public SparkJoinHintOptimizer(ParseContext parseCtx) { } @Override - public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException { - MapJoinOperator mapJoinOp = (MapJoinOperator) nd; + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { OptimizeSparkProcContext context = (OptimizeSparkProcContext) procCtx; HiveConf hiveConf = context.getParseContext().getConf(); // Convert from mapjoin to bucket map join if enabled. - if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVEOPTBUCKETMAPJOIN) || hiveConf.getBoolVar(HiveConf.ConfVars.HIVEOPTSORTMERGEBUCKETMAPJOIN)) { + if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVEOPTBUCKETMAPJOIN) + || hiveConf.getBoolVar(HiveConf.ConfVars.HIVEOPTSORTMERGEBUCKETMAPJOIN)) { BucketJoinProcCtx bjProcCtx = new BucketJoinProcCtx(hiveConf); bucketMapJoinOptimizer.process(nd, stack, bjProcCtx, nodeOutputs); } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java index 51bdaf4..4645c09 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java @@ -29,7 +29,8 @@ import java.util.Stack; /** - * Converts a join to a more optimized join for the Spark path. Delegates to a more specialized join processor. + * Converts a join to a more optimized join for the Spark path. + * Delegates to a more specialized join processor. */ public class SparkJoinOptimizer implements NodeProcessor { diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java index 80665ec..b878120 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java @@ -60,8 +60,8 @@ private static final Log LOG = LogFactory.getLog(SparkMapJoinOptimizer.class.getName()); @Override - /* - * (non-Javadoc) we should ideally not modify the tree we traverse. However, + /** + * We should ideally not modify the tree we traverse. However, * since we need to walk the tree at any time when we modify the operator, we * might as well do it here. */ @@ -74,66 +74,14 @@ JoinOperator joinOp = (JoinOperator) nd; if (!conf.getBoolVar(HiveConf.ConfVars.HIVECONVERTJOIN)) { - // && !(conf.getBoolVar(HiveConf.ConfVars.HIVE_AUTO_SORTMERGE_JOIN))) { - // we are just converting to a common merge join operator. The shuffle - // join in map-reduce case. - // int pos = 0; // it doesn't matter which position we use in this case. - // convertJoinSMBJoin(joinOp, context, pos, 0, false, false); return null; } LOG.info("Check if it can be converted to map join"); long[] mapJoinInfo = getMapJoinConversionInfo(joinOp, context); - int mapJoinConversionPos = (int)mapJoinInfo[0]; + int mapJoinConversionPos = (int) mapJoinInfo[0]; if (mapJoinConversionPos < 0) { - /* TODO: handle this later - // we cannot convert to bucket map join, we cannot convert to - // map join either based on the size. Check if we can convert to SMB join. - if (conf.getBoolVar(HiveConf.ConfVars.HIVE_AUTO_SORTMERGE_JOIN) == false) { - convertJoinSMBJoin(joinOp, context, 0, 0, false, false); - return null; - } - Class bigTableMatcherClass = null; - try { - bigTableMatcherClass = - (Class) (Class.forName(HiveConf.getVar( - parseContext.getConf(), - HiveConf.ConfVars.HIVE_AUTO_SORTMERGE_JOIN_BIGTABLE_SELECTOR))); - } catch (ClassNotFoundException e) { - throw new SemanticException(e.getMessage()); - } - - BigTableSelectorForAutoSMJ bigTableMatcher = - ReflectionUtils.newInstance(bigTableMatcherClass, null); - JoinDesc joinDesc = joinOp.getConf(); - JoinCondDesc[] joinCondns = joinDesc.getConds(); - Set joinCandidates = MapJoinProcessor.getBigTableCandidates(joinCondns); - if (joinCandidates.isEmpty()) { - // This is a full outer join. This can never be a map-join - // of any type. So return false. - return false; - } - mapJoinConversionPos = - bigTableMatcher.getBigTablePosition(parseContext, joinOp, joinCandidates); - if (mapJoinConversionPos < 0) { - // contains aliases from sub-query - // we are just converting to a common merge join operator. The shuffle - // join in map-reduce case. - int pos = 0; // it doesn't matter which position we use in this case. - convertJoinSMBJoin(joinOp, context, pos, 0, false, false); - return null; - } - - if (checkConvertJoinSMBJoin(joinOp, context, mapJoinConversionPos, tezBucketJoinProcCtx)) { - convertJoinSMBJoin(joinOp, context, mapJoinConversionPos, - tezBucketJoinProcCtx.getNumBuckets(), tezBucketJoinProcCtx.isSubQuery(), true); - } else { - // we are just converting to a common merge join operator. The shuffle - // join in map-reduce case. - int pos = 0; // it doesn't matter which position we use in this case. - convertJoinSMBJoin(joinOp, context, pos, 0, false, false); - } */ return null; } @@ -166,107 +114,9 @@ return mapJoinOp; } - // replaces the join operator with a new CommonJoinOperator, removes the - // parent reduce sinks - /* - private void convertJoinSMBJoin(JoinOperator joinOp, OptimizeSparkProcContext context, - int mapJoinConversionPos, int numBuckets, boolean isSubQuery, boolean adjustParentsChildren) - throws SemanticException { - ParseContext parseContext = context.parseContext; - MapJoinDesc mapJoinDesc = null; - if (adjustParentsChildren) { - mapJoinDesc = MapJoinProcessor.getMapJoinDesc(context.conf, parseContext.getOpParseCtx(), - joinOp, parseContext.getJoinContext().get(joinOp), mapJoinConversionPos, true); - } else { - JoinDesc joinDesc = joinOp.getConf(); - // retain the original join desc in the map join. - mapJoinDesc = - new MapJoinDesc(null, null, joinDesc.getExprs(), null, null, - joinDesc.getOutputColumnNames(), mapJoinConversionPos, joinDesc.getConds(), - joinDesc.getFilters(), joinDesc.getNoOuterJoin(), null); - } - - @SuppressWarnings("unchecked") - CommonMergeJoinOperator mergeJoinOp = - (CommonMergeJoinOperator) OperatorFactory.get(new CommonMergeJoinDesc(numBuckets, - isSubQuery, mapJoinConversionPos, mapJoinDesc)); - OpTraits opTraits = - new OpTraits(joinOp.getOpTraits().getBucketColNames(), numBuckets, joinOp.getOpTraits() - .getSortCols()); - mergeJoinOp.setOpTraits(opTraits); - mergeJoinOp.setStatistics(joinOp.getStatistics()); - - for (Operator parentOp : joinOp.getParentOperators()) { - int pos = parentOp.getChildOperators().indexOf(joinOp); - parentOp.getChildOperators().remove(pos); - parentOp.getChildOperators().add(pos, mergeJoinOp); - } - - for (Operator childOp : joinOp.getChildOperators()) { - int pos = childOp.getParentOperators().indexOf(joinOp); - childOp.getParentOperators().remove(pos); - childOp.getParentOperators().add(pos, mergeJoinOp); - } - - List> childOperators = mergeJoinOp.getChildOperators(); - if (childOperators == null) { - childOperators = new ArrayList>(); - mergeJoinOp.setChildOperators(childOperators); - } - - List> parentOperators = mergeJoinOp.getParentOperators(); - if (parentOperators == null) { - parentOperators = new ArrayList>(); - mergeJoinOp.setParentOperators(parentOperators); - } - - childOperators.clear(); - parentOperators.clear(); - childOperators.addAll(joinOp.getChildOperators()); - parentOperators.addAll(joinOp.getParentOperators()); - mergeJoinOp.getConf().setGenJoinKeys(false); - - if (adjustParentsChildren) { - mergeJoinOp.getConf().setGenJoinKeys(true); - List> newParentOpList = - new ArrayList>(); - for (Operator parentOp : mergeJoinOp.getParentOperators()) { - for (Operator grandParentOp : parentOp.getParentOperators()) { - grandParentOp.getChildOperators().remove(parentOp); - grandParentOp.getChildOperators().add(mergeJoinOp); - newParentOpList.add(grandParentOp); - } - } - mergeJoinOp.getParentOperators().clear(); - mergeJoinOp.getParentOperators().addAll(newParentOpList); - List> parentOps = - new ArrayList>(mergeJoinOp.getParentOperators()); - for (Operator parentOp : parentOps) { - int parentIndex = mergeJoinOp.getParentOperators().indexOf(parentOp); - if (parentIndex == mapJoinConversionPos) { - continue; - } - - // insert the dummy store operator here - DummyStoreOperator dummyStoreOp = new TezDummyStoreOperator(); - dummyStoreOp.setParentOperators(new ArrayList>()); - dummyStoreOp.setChildOperators(new ArrayList>()); - dummyStoreOp.getChildOperators().add(mergeJoinOp); - int index = parentOp.getChildOperators().indexOf(mergeJoinOp); - parentOp.getChildOperators().remove(index); - parentOp.getChildOperators().add(index, dummyStoreOp); - dummyStoreOp.getParentOperators().add(parentOp); - mergeJoinOp.getParentOperators().remove(parentIndex); - mergeJoinOp.getParentOperators().add(parentIndex, dummyStoreOp); - } - } - mergeJoinOp.cloneOriginalParentsList(mergeJoinOp.getParentOperators()); - } - */ - private void setNumberOfBucketsOnChildren(Operator currentOp) { int numBuckets = currentOp.getOpTraits().getNumBuckets(); - for (Operatorop : currentOp.getChildOperators()) { + for (Operator op : currentOp.getChildOperators()) { if (!(op instanceof ReduceSinkOperator) && !(op instanceof GroupByOperator)) { op.getOpTraits().setNumBuckets(numBuckets); if (numBuckets < 0) { @@ -298,8 +148,8 @@ private int convertJoinBucketMapJoin(JoinOperator joinOp, MapJoinOperator mapJoi BucketMapjoinProc.checkAndConvertBucketMapJoin( parseContext, mapJoinOp, joinTree, baseBigAlias, joinAliases); MapJoinDesc joinDesc = mapJoinOp.getConf(); - return joinDesc.isBucketMapJoin() ? - joinDesc.getBigTableBucketNumMapping().size() : -1; + return joinDesc.isBucketMapJoin() + ? joinDesc.getBigTableBucketNumMapping().size() : -1; } /** @@ -337,7 +187,7 @@ private int convertJoinBucketMapJoin(JoinOperator joinOp, MapJoinOperator mapJoi Statistics currInputStat = parentOp.getStatistics(); if (currInputStat == null) { - LOG.warn("Couldn't get statistics from: "+parentOp); + LOG.warn("Couldn't get statistics from: " + parentOp); return new long[]{-1, 0, 0}; } @@ -359,15 +209,14 @@ private int convertJoinBucketMapJoin(JoinOperator joinOp, MapJoinOperator mapJoi // Otherwise, we could try to break the op tree at the UNION, and create two MapWorks // for the branches above. Then, MJ will be in the following ReduceWork. // But, this is tricky to implement, and we'll leave it as a future work for now. - // TODO: handle this as a MJ case if (containUnionWithoutRS(parentOp.getParentOperators().get(0))) { return new long[]{-1, 0, 0}; } long inputSize = currInputStat.getDataSize(); - if ((bigInputStat == null) || - ((bigInputStat != null) && - (inputSize > bigInputStat.getDataSize()))) { + if ((bigInputStat == null) + || ((bigInputStat != null) + && (inputSize > bigInputStat.getDataSize()))) { if (bigTableFound) { // cannot convert to map join; we've already chosen a big table @@ -416,9 +265,10 @@ private int convertJoinBucketMapJoin(JoinOperator joinOp, MapJoinOperator mapJoi return new long[]{-1, 0, 0}; } - //Final check, find size of already-calculated Mapjoin Operators in same work (spark-stage). We need to factor - //this in to prevent overwhelming Spark executor-memory. - long connectedMapJoinSize = getConnectedMapJoinSize(joinOp.getParentOperators().get(bigTablePosition), joinOp, context); + //Final check, find size of already-calculated Mapjoin Operators in same work (spark-stage). + //We need to factor this in to prevent overwhelming Spark executor-memory. + long connectedMapJoinSize = getConnectedMapJoinSize(joinOp.getParentOperators(). + get(bigTablePosition), joinOp, context); if ((connectedMapJoinSize + totalSize) > maxSize) { return new long[]{-1, 0, 0}; } @@ -434,7 +284,8 @@ private int convertJoinBucketMapJoin(JoinOperator joinOp, MapJoinOperator mapJoi * @return total size of parent mapjoins in same work as this operator. */ @SuppressWarnings({"rawtypes", "unchecked"}) - private long getConnectedMapJoinSize(Operator parentOp, Operator joinOp, OptimizeSparkProcContext ctx) { + private long getConnectedMapJoinSize(Operator parentOp, Operator joinOp, + OptimizeSparkProcContext ctx) { long result = 0; for (Operator grandParentOp : parentOp.getParentOperators()) { result += getConnectedParentMapJoinSize(grandParentOp, ctx); @@ -482,7 +333,8 @@ private long getConnectedChildMapJoinSize(Operator op, O } if (op instanceof MapJoinOperator) { - //found child mapjoin operator. Its size should already reflect any mapjoins connected to it, so stop processing. + //Found child mapjoin operator. + //Its size should already reflect any mapjoins connected to it, so stop processing. long mjSize = ctx.getMjOpSizes().get(op); return mjSize; } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java index 113832f..918c7d6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.optimizer.spark; import com.google.common.base.Preconditions; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; @@ -60,6 +61,8 @@ import java.util.Stack; public class SparkReduceSinkMapJoinProc implements NodeProcessor { + + public static final Log LOG = LogFactory.getLog(SparkReduceSinkMapJoinProc.class.getName()); public static class SparkMapJoinFollowedByGroupByProcessor implements NodeProcessor { private boolean hasGroupBy = false; @@ -81,8 +84,6 @@ public boolean getHasGroupBy() { } } - protected transient Log LOG = LogFactory.getLog(this.getClass().getName()); - private boolean hasGroupBy(Operator mapjoinOp, GenSparkProcContext context) throws SemanticException { List> childOps = mapjoinOp.getChildOperators(); @@ -106,6 +107,7 @@ private boolean hasGroupBy(Operator mapjoinOp, * on the basis of the big table side because it may be a mapwork (no need for shuffle) * or reduce work. */ + @SuppressWarnings("unchecked") @Override public Object process(Node nd, Stack stack, NodeProcessorCtx procContext, Object... nodeOutputs) @@ -132,14 +134,14 @@ public Object process(Node nd, Stack stack, parentRS.setSkipTag(true); // remember the original parent list before we start modifying it. if (!context.mapJoinParentMap.containsKey(mapJoinOp)) { - List> parents = new ArrayList(mapJoinOp.getParentOperators()); + List> parents = new ArrayList>(mapJoinOp.getParentOperators()); context.mapJoinParentMap.put(mapJoinOp, parents); } List mapJoinWork; /* - * if there was a pre-existing work generated for the big-table mapjoin side, + * If there was a pre-existing work generated for the big-table mapjoin side, * we need to hook the work generated for the RS (associated with the RS-MJ pattern) * with the pre-existing work. * @@ -161,20 +163,6 @@ public Object process(Node nd, Stack stack, LOG.debug("Mapjoin "+mapJoinOp+", pos: "+pos+" --> "+parentWork.getName()); mapJoinOp.getConf().getParentToInput().put(pos, parentWork.getName()); -/* int numBuckets = -1; - EdgeType edgeType = EdgeType.BROADCAST_EDGE; - if (mapJoinOp.getConf().isBucketMapJoin()) { - - // disable auto parallelism for bucket map joins - parentRS.getConf().setAutoParallel(false); - - numBuckets = (Integer) mapJoinOp.getConf().getBigTableBucketNumMapping().values().toArray()[0]; - if (mapJoinOp.getConf().getCustomBucketMapJoin()) { - edgeType = EdgeType.CUSTOM_EDGE; - } else { - edgeType = EdgeType.CUSTOM_SIMPLE_EDGE; - } - }*/ SparkEdgeProperty edgeProp = new SparkEdgeProperty(SparkEdgeProperty.SHUFFLE_NONE); if (mapJoinWork != null) { @@ -209,7 +197,6 @@ public Object process(Node nd, Stack stack, // create an new operator: HashTableDummyOperator, which share the table desc HashTableDummyDesc desc = new HashTableDummyDesc(); - @SuppressWarnings("unchecked") HashTableDummyOperator dummyOp = (HashTableDummyOperator) OperatorFactory.get(desc); TableDesc tbl; @@ -221,7 +208,7 @@ public Object process(Node nd, Stack stack, Map> keyExprMap = mapJoinOp.getConf().getKeys(); List keyCols = keyExprMap.get(Byte.valueOf((byte) 0)); StringBuffer keyOrder = new StringBuffer(); - for (ExprNodeDesc k: keyCols) { + for (int i = 0; i < keyCols.size(); i++) { keyOrder.append("+"); } TableDesc keyTableDesc = PlanUtils.getReduceKeyTableDesc(PlanUtils @@ -291,11 +278,11 @@ public Object process(Node nd, Stack stack, } //get all parents of reduce sink - List> RSparentOps = parentRS.getParentOperators(); - for (Operator parent : RSparentOps) { + List> rsParentOps = parentRS.getParentOperators(); + for (Operator parent : rsParentOps) { parent.replaceChild(parentRS, hashTableSinkOp); } - hashTableSinkOp.setParentOperators(RSparentOps); + hashTableSinkOp.setParentOperators(rsParentOps); hashTableSinkOp.setTag(tag); return true; } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSMBJoinHintOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSMBJoinHintOptimizer.java index 7b69cb7..59a5e89 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSMBJoinHintOptimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSMBJoinHintOptimizer.java @@ -1,12 +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. + */ + package org.apache.hadoop.hive.ql.optimizer.spark; -import com.clearspring.analytics.util.Preconditions; +import java.util.List; +import java.util.Stack; + import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.lib.NodeProcessor; @@ -18,8 +37,7 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.SMBJoinDesc; -import java.util.List; -import java.util.Stack; +import com.clearspring.analytics.util.Preconditions; /** * Converts from a bucket-mapjoin created from hints to SMB mapjoin. @@ -44,8 +62,8 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, // Throw an error if the user asked for sort merge bucketed mapjoin to be enforced // and sort merge bucketed mapjoin cannot be performed - if (!convert && - pGraphContext.getConf().getBoolVar( + if (!convert + && pGraphContext.getConf().getBoolVar( HiveConf.ConfVars.HIVEENFORCESORTMERGEBUCKETMAPJOIN)) { throw new SemanticException(ErrorMsg.SORTMERGE_MAPJOIN_FAILED.getMsg()); } @@ -62,6 +80,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, * In SMB join these are not expected for any parents, either from small or big tables. * @param mapJoinOp */ + @SuppressWarnings("unchecked") private void removeSmallTableReduceSink(MapJoinOperator mapJoinOp) { SMBJoinDesc smbJoinDesc = new SMBJoinDesc(mapJoinOp.getConf()); List> parentOperators = mapJoinOp.getParentOperators(); diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java index 77a447e..4eed8ea 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java @@ -56,7 +56,7 @@ import java.util.Stack; /** - * Spark-version of SkewJoinProcFactory + * Spark-version of SkewJoinProcFactory. */ public class SparkSkewJoinProcFactory { private SparkSkewJoinProcFactory() { @@ -82,10 +82,10 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, JoinOperator op = (JoinOperator) nd; ReduceWork reduceWork = context.getReducerToReduceWork().get(op); ParseContext parseContext = context.getParseCtx(); - if (!op.getConf().isFixedAsSorted() && currentTsk instanceof SparkTask && - reduceWork != null && ((SparkTask) currentTsk).getWork().contains(reduceWork) && - GenSparkSkewJoinProcessor.supportRuntimeSkewJoin( - op, currentTsk, parseContext.getConf())) { + if (!op.getConf().isFixedAsSorted() && currentTsk instanceof SparkTask + && reduceWork != null && ((SparkTask) currentTsk).getWork().contains(reduceWork) + && GenSparkSkewJoinProcessor.supportRuntimeSkewJoin( + op, currentTsk, parseContext.getConf())) { // first we try to split the task splitTask((SparkTask) currentTsk, reduceWork, parseContext); GenSparkSkewJoinProcessor.processSkewJoin(op, currentTsk, reduceWork, parseContext); @@ -102,8 +102,8 @@ private static void splitTask(SparkTask currentTask, ReduceWork reduceWork, SparkWork currentWork = currentTask.getWork(); Set> reduceSinkSet = SparkMapJoinResolver.getOp(reduceWork, ReduceSinkOperator.class); - if (currentWork.getChildren(reduceWork).size() == 1 && canSplit(currentWork) && - reduceSinkSet.size() == 1) { + if (currentWork.getChildren(reduceWork).size() == 1 && canSplit(currentWork) + && reduceSinkSet.size() == 1) { ReduceSinkOperator reduceSink = (ReduceSinkOperator) reduceSinkSet.iterator().next(); BaseWork childWork = currentWork.getChildren(reduceWork).get(0); SparkEdgeProperty originEdge = currentWork.getEdgeProperty(reduceWork, childWork); @@ -118,7 +118,6 @@ private static void splitTask(SparkTask currentTask, ReduceWork reduceWork, // remove them from current spark work for (BaseWork baseWork : newWork.getAllWorkUnsorted()) { currentWork.remove(baseWork); - // TODO: take care of cloneToWork? currentWork.getCloneToWork().remove(baseWork); } // create TS to read intermediate data @@ -152,7 +151,6 @@ private static void splitTask(SparkTask currentTask, ReduceWork reduceWork, } else { streamDesc = "$INTNAME"; } - // TODO: remove this? String origStreamDesc = streamDesc; int pos = 0; while (mapWork.getAliasToWork().get(streamDesc) != null) { @@ -162,6 +160,7 @@ private static void splitTask(SparkTask currentTask, ReduceWork reduceWork, GenMapRedUtils.setTaskPlan(taskTmpDir.toUri().toString(), streamDesc, tableScanOp, mapWork, false, tableDesc); // insert the new task between current task and its child + @SuppressWarnings("unchecked") Task newTask = TaskFactory.get(newWork, parseContext.getConf()); List> childTasks = currentTask.getChildTasks(); // must have at most one child @@ -190,7 +189,7 @@ private static boolean canSplit(SparkWork sparkWork) { } /** - * Copy a sub-graph from originWork to newWork + * Copy a sub-graph from originWork to newWork. */ private static void copyWorkGraph(SparkWork originWork, SparkWork newWork, BaseWork baseWork, boolean upWards) { diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java index 354c78c..984380d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java @@ -18,8 +18,16 @@ package org.apache.hadoop.hive.ql.optimizer.spark; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Stack; + import org.apache.hadoop.hive.ql.exec.CommonJoinOperator; -import org.apache.hadoop.hive.ql.exec.ConditionalTask; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.spark.SparkTask; @@ -39,17 +47,8 @@ import org.apache.hadoop.hive.ql.plan.ReduceWork; import org.apache.hadoop.hive.ql.plan.SparkWork; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Stack; - /** - * Spark version of SkewJoinResolver + * Spark version of SkewJoinResolver. */ public class SparkSkewJoinResolver implements PhysicalPlanResolver { @Override @@ -62,7 +61,7 @@ public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { return pctx; } - class SparkSkewJoinTaskDispatcher implements Dispatcher{ + class SparkSkewJoinTaskDispatcher implements Dispatcher { private PhysicalContext physicalContext; public SparkSkewJoinTaskDispatcher(PhysicalContext context) { @@ -74,6 +73,7 @@ public SparkSkewJoinTaskDispatcher(PhysicalContext context) { public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) throws SemanticException { + @SuppressWarnings("unchecked") Task task = (Task) nd; if (task instanceof SparkTask) { SparkWork sparkWork = ((SparkTask) task).getWork(); diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinFactory.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinFactory.java index 3cee91e..6e0ac38 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinFactory.java @@ -17,10 +17,13 @@ */ package org.apache.hadoop.hive.ql.optimizer.spark; +import java.util.List; +import java.util.Map; +import java.util.Stack; + import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator; import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.lib.NodeProcessor; import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; @@ -32,10 +35,6 @@ import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import java.util.List; -import java.util.Map; -import java.util.Stack; - /** * Operator factory for Spark SMBJoin processing. */ @@ -53,6 +52,7 @@ public static int getPositionParent(SMBMapJoinOperator op, Stack stack) { int size = stack.size(); assert size >= 2 && stack.get(size - 1) == op; + @SuppressWarnings("unchecked") Operator parent = (Operator) stack.get(size - 2); List> parOp = op.getParentOperators(); @@ -154,8 +154,6 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, SMBMapJoinOperator mapJoin = (SMBMapJoinOperator) nd; GenSparkProcContext ctx = (GenSparkProcContext) procCtx; - SparkTask currTask = ctx.currentTask; - // find the branch on which this processor was invoked int pos = getPositionParent(mapJoin, stack); boolean local = pos != mapJoin.getConf().getPosBigTable(); diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java index fc8021c..b739682 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java @@ -70,26 +70,29 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, return null; } - protected boolean canConvertJoinToSMBJoin(JoinOperator joinOperator, SortBucketJoinProcCtx smbJoinContext, - ParseContext pGraphContext, Stack stack) throws SemanticException { + protected boolean canConvertJoinToSMBJoin(JoinOperator joinOperator, + SortBucketJoinProcCtx smbJoinContext, ParseContext pGraphContext, + Stack stack) throws SemanticException { if (!supportBucketMapJoin(stack)) { return false; } return canConvertJoinToSMBJoin(joinOperator, smbJoinContext, pGraphContext); } - //Preliminary checks. In the MR version of the code, these used to be done via another walk, refactoring to be inline. + //Preliminary checks. In the MR version of the code, these used to be done via another walk, + //here it is done inline. private boolean supportBucketMapJoin(Stack stack) { int size = stack.size(); - if (!(stack.get(size-1) instanceof JoinOperator) || - !(stack.get(size-2) instanceof ReduceSinkOperator)) { + if (!(stack.get(size - 1) instanceof JoinOperator) + || !(stack.get(size - 2) instanceof ReduceSinkOperator)) { return false; } // If any operator in the stack does not support a auto-conversion, this join should // not be converted. - for (int pos = size -3; pos >= 0; pos--) { - Operator op = (Operator)stack.get(pos); + for (int pos = size - 3; pos >= 0; pos--) { + @SuppressWarnings("unchecked") + Operator op = (Operator) stack.get(pos); if (!op.supportAutomaticSortMergeJoin()) { return false; } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SplitSparkWorkResolver.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SplitSparkWorkResolver.java index 67dda02..ed30e76 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SplitSparkWorkResolver.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SplitSparkWorkResolver.java @@ -18,7 +18,16 @@ package org.apache.hadoop.hive.ql.optimizer.spark; -import com.google.common.base.Preconditions; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; + import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; import org.apache.hadoop.hive.ql.exec.Task; @@ -35,8 +44,8 @@ import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; import org.apache.hadoop.hive.ql.plan.SparkWork; -import java.io.Serializable; -import java.util.*; +import com.google.common.base.Preconditions; + /** * Do a BFS on the sparkWork graph, and look for any work that has more than one child. @@ -154,13 +163,12 @@ private void removeOpRecursive(Operator operator) { } // we lost statistics & opTraits through cloning, try to get them back - // TODO: make sure this method is sufficient to solve the problem private void setStatistics(BaseWork origin, BaseWork clone) { if (origin instanceof MapWork && clone instanceof MapWork) { MapWork originMW = (MapWork) origin; MapWork cloneMW = (MapWork) clone; - for (Map.Entry> entry : - originMW.getAliasToWork().entrySet()) { + for (Map.Entry> entry + : originMW.getAliasToWork().entrySet()) { String alias = entry.getKey(); Operator cloneOP = cloneMW.getAliasToWork().get(alias); if (cloneOP != null) { diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java index 778a5ab..dcd9018 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java @@ -58,7 +58,7 @@ * Cloned from GenTezProcContext. * */ -public class GenSparkProcContext implements NodeProcessorCtx{ +public class GenSparkProcContext implements NodeProcessorCtx { public final ParseContext parseContext; public final HiveConf conf; public final List> moveTask; @@ -89,11 +89,12 @@ // map that keeps track of the last operator of a task to the following work // of this operator. This is used for connecting them later. - public final Map> leafOpToFollowingWorkInfo; + public final Map> + leafOpToFollowingWorkInfo; // a map that keeps track of work that need to be linked while // traversing an operator tree - public final Map, Map> linkOpWithWorkMap; + public final Map, Map> linkOpWithWorkMap; // a map to keep track of what reduce sinks have to be hooked up to // map join work @@ -138,9 +139,13 @@ public final Map> topOps; @SuppressWarnings("unchecked") - public GenSparkProcContext(HiveConf conf, ParseContext parseContext, - List> moveTask, List> rootTasks, - Set inputs, Set outputs, Map> topOps) { + public GenSparkProcContext(HiveConf conf, + ParseContext parseContext, + List> moveTask, + List> rootTasks, + Set inputs, + Set outputs, + Map> topOps) { this.conf = conf; this.parseContext = parseContext; this.moveTask = moveTask; @@ -163,9 +168,9 @@ public GenSparkProcContext(HiveConf conf, ParseContext parseContext, this.currentMapJoinOperators = new LinkedHashSet(); this.linkChildOpWithDummyOp = new LinkedHashMap, List>>(); this.dependencyTask = conf.getBoolVar( - HiveConf.ConfVars.HIVE_MULTI_INSERT_MOVE_TASKS_SHARE_DEPENDENCIES) ? - (DependencyCollectionTask) TaskFactory.get(new DependencyCollectionWork(), conf) : - null; + HiveConf.ConfVars.HIVE_MULTI_INSERT_MOVE_TASKS_SHARE_DEPENDENCIES) + ? (DependencyCollectionTask) TaskFactory.get(new DependencyCollectionWork(), conf) + : null; this.unionWorkMap = new LinkedHashMap, BaseWork>(); this.currentUnionOperators = new LinkedList(); this.workWithUnionOperators = new LinkedHashSet(); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java index f498e9e..6e8f090 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java @@ -18,14 +18,21 @@ package org.apache.hadoop.hive.ql.parse.spark; -import com.google.common.base.Preconditions; -import com.google.common.base.Strings; +import java.util.ArrayList; +import java.util.Deque; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; 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.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.exec.FetchTask; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; @@ -48,26 +55,16 @@ import org.apache.hadoop.hive.ql.plan.ReduceWork; import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.plan.UnionWork; -import org.apache.hadoop.hive.ql.stats.StatsFactory; -import java.util.ArrayList; -import java.util.Deque; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; /** * GenSparkUtils is a collection of shared helper methods to produce SparkWork * Cloned from GenTezUtils. - * TODO: need to make it fit to Spark */ public class GenSparkUtils { - private static final Log logger = LogFactory.getLog(GenSparkUtils.class.getName()); + private static final Log LOG = LogFactory.getLog(GenSparkUtils.class.getName()); // sequence number is used to name vertices (e.g.: Map 1, Reduce 14, ...) private int sequenceNumber = 0; @@ -89,12 +86,13 @@ public void resetSequenceNumber() { sequenceNumber = 0; } - public ReduceWork createReduceWork(GenSparkProcContext context, Operator root, SparkWork sparkWork) throws SemanticException { + public ReduceWork createReduceWork(GenSparkProcContext context, Operator root, + SparkWork sparkWork) throws SemanticException { Preconditions.checkArgument(!root.getParentOperators().isEmpty(), "AssertionError: expected root.getParentOperators() to be non-empty"); - ReduceWork reduceWork = new ReduceWork("Reducer "+ (++sequenceNumber)); - logger.debug("Adding reduce work (" + reduceWork.getName() + ") for " + root); + ReduceWork reduceWork = new ReduceWork("Reducer " + (++sequenceNumber)); + LOG.debug("Adding reduce work (" + reduceWork.getName() + ") for " + root); reduceWork.setReducer(root); reduceWork.setNeedsTagging(GenMapRedUtils.needsTagging(reduceWork)); @@ -103,8 +101,8 @@ public ReduceWork createReduceWork(GenSparkProcContext context, Operator root // all be -1. In sort/order case where it matters there will be only // one parent. Preconditions.checkArgument(context.parentOfRoot instanceof ReduceSinkOperator, - "AssertionError: expected context.parentOfRoot to be an instance of ReduceSinkOperator, but was " + - context.parentOfRoot.getClass().getName()); + "AssertionError: expected context.parentOfRoot to be an instance of ReduceSinkOperator, but was " + + context.parentOfRoot.getClass().getName()); ReduceSinkOperator reduceSink = (ReduceSinkOperator) context.parentOfRoot; reduceWork.setNumReduceTasks(reduceSink.getConf().getNumReducers()); @@ -123,7 +121,7 @@ public ReduceWork createReduceWork(GenSparkProcContext context, Operator root protected void setupReduceSink(GenSparkProcContext context, ReduceWork reduceWork, ReduceSinkOperator reduceSink) { - logger.debug("Setting up reduce sink: " + reduceSink + LOG.debug("Setting up reduce sink: " + reduceSink + " with following reduce work: " + reduceWork.getName()); // need to fill in information about the key and value in the reducer @@ -146,14 +144,14 @@ public MapWork createMapWork(GenSparkProcContext context, Operator root, SparkWork sparkWork, PrunedPartitionList partitions, boolean deferSetup) throws SemanticException { Preconditions.checkArgument(root.getParentOperators().isEmpty(), "AssertionError: expected root.getParentOperators() to be empty"); - MapWork mapWork = new MapWork("Map "+ (++sequenceNumber)); - logger.debug("Adding map work (" + mapWork.getName() + ") for " + root); + MapWork mapWork = new MapWork("Map " + (++sequenceNumber)); + LOG.debug("Adding map work (" + mapWork.getName() + ") for " + root); // map work starts with table scan operators Preconditions.checkArgument(root instanceof TableScanOperator, - "AssertionError: expected root to be an instance of TableScanOperator, but was " + - root.getClass().getName()); - String alias = ((TableScanOperator)root).getConf().getAlias(); + "AssertionError: expected root to be an instance of TableScanOperator, but was " + + root.getClass().getName()); + String alias = ((TableScanOperator) root).getConf().getAlias(); if (!deferSetup) { setupMapWork(mapWork, context, partitions, root, alias); @@ -174,11 +172,11 @@ protected void setupMapWork(MapWork mapWork, GenSparkProcContext context, context.inputs, partitions, root, alias, context.conf, false); } - private void collectOperators (Operator op, List> opList) { + private void collectOperators(Operator op, List> opList) { opList.add(op); for (Object child : op.getChildOperators()) { if (child != null) { - collectOperators((Operator)child, opList); + collectOperators((Operator) child, opList); } } } @@ -199,23 +197,23 @@ public void removeUnionOperators(Configuration conf, GenSparkProcContext context // Build a map to map the original FileSinkOperator and the cloned FileSinkOperators // This map is used for set the stats flag for the cloned FileSinkOperators in later process - Iterator> newRoots_it = newRoots.iterator(); + Iterator> newRootsIt = newRoots.iterator(); for (Operator root : roots) { - Operator newRoot = newRoots_it.next(); + Operator newRoot = newRootsIt.next(); List> newOpQueue = new LinkedList>(); - collectOperators (newRoot, newOpQueue); + collectOperators(newRoot, newOpQueue); List> opQueue = new LinkedList>(); - collectOperators (root, opQueue); - Iterator> newOpQueue_it = newOpQueue.iterator(); + collectOperators(root, opQueue); + Iterator> newOpQueueIt = newOpQueue.iterator(); for (Operator op : opQueue) { - Operator newOp = newOpQueue_it.next(); + Operator newOp = newOpQueueIt.next(); if (op instanceof FileSinkOperator) { List fileSinkList = context.fileSinkMap.get(op); if (fileSinkList == null) { fileSinkList = new LinkedList(); } - fileSinkList.add((FileSinkOperator)newOp); - context.fileSinkMap.put((FileSinkOperator)op, fileSinkList); + fileSinkList.add((FileSinkOperator) newOp); + context.fileSinkMap.put((FileSinkOperator) op, fileSinkList); } } } @@ -234,10 +232,10 @@ public void removeUnionOperators(Configuration conf, GenSparkProcContext context for (Operator orig: roots) { Operator newRoot = it.next(); if (newRoot instanceof HashTableDummyOperator) { - dummyOps.add((HashTableDummyOperator)newRoot); + dummyOps.add((HashTableDummyOperator) newRoot); it.remove(); } else { - replacementMap.put(orig,newRoot); + replacementMap.put(orig, newRoot); } } @@ -249,7 +247,7 @@ public void removeUnionOperators(Configuration conf, GenSparkProcContext context Set> seen = new HashSet>(); - while(!operators.isEmpty()) { + while (!operators.isEmpty()) { Operator current = operators.pop(); seen.add(current); @@ -314,7 +312,7 @@ public void processFileSink(GenSparkProcContext context, FileSinkOperator fileSi if (chDir) { // Merge the files in the destination table/partitions by creating Map-only merge job // If underlying data is RCFile a RCFileBlockMerge task would be created. - logger.info("using CombineHiveInputformat for the merge job"); + LOG.info("using CombineHiveInputformat for the merge job"); GenMapRedUtils.createMRWorkForMergingFiles(fileSink, finalName, context.dependencyTask, context.moveTask, hconf, context.currentTask); @@ -335,8 +333,8 @@ public static SparkEdgeProperty getEdgeProperty(ReduceSinkOperator reduceSink, String sortOrder = Strings.nullToEmpty(reduceSink.getConf().getOrder()).trim(); // test if we need group-by shuffle - if (reduceSink.getChildOperators().size() == 1 && - reduceSink.getChildOperators().get(0) instanceof GroupByOperator) { + if (reduceSink.getChildOperators().size() == 1 + && reduceSink.getChildOperators().get(0) instanceof GroupByOperator) { edgeProperty.setShuffleGroup(); // test if the group by needs partition level sort, if so, use the MR style shuffle // SHUFFLE_SORT shouldn't be used for this purpose, see HIVE-8542 @@ -364,9 +362,9 @@ public static SparkEdgeProperty getEdgeProperty(ReduceSinkOperator reduceSink, // test if we need total order, if so, we can either use MR shuffle + set #reducer to 1, // or we can use SHUFFLE_SORT if (edgeProperty.isShuffleNone() && !sortOrder.isEmpty()) { - if (reduceSink.getConf().getPartitionCols() == null || - reduceSink.getConf().getPartitionCols().isEmpty() || - isSame(reduceSink.getConf().getPartitionCols(), + if (reduceSink.getConf().getPartitionCols() == null + || reduceSink.getConf().getPartitionCols().isEmpty() + || isSame(reduceSink.getConf().getPartitionCols(), reduceSink.getConf().getKeyCols())) { edgeProperty.setShuffleSort(); } else { @@ -397,12 +395,12 @@ private static boolean groupByNeedParLevelOrder(ReduceSinkOperator reduceSinkOpe return true; } List> children = reduceSinkOperator.getChildOperators(); - if (children != null && children.size() == 1 && - children.get(0) instanceof GroupByOperator) { + if (children != null && children.size() == 1 + && children.get(0) instanceof GroupByOperator) { GroupByOperator child = (GroupByOperator) children.get(0); if (isSame(reduceSinkOperator.getConf().getKeyCols(), - reduceSinkOperator.getConf().getPartitionCols()) && - reduceSinkOperator.getConf().getKeyCols().size() == child.getConf().getKeys().size()) { + reduceSinkOperator.getConf().getPartitionCols()) + && reduceSinkOperator.getConf().getKeyCols().size() == child.getConf().getKeys().size()) { return false; } } @@ -410,7 +408,7 @@ private static boolean groupByNeedParLevelOrder(ReduceSinkOperator reduceSinkOpe } /** - * Test if two lists of ExprNodeDesc are semantically same + * Test if two lists of ExprNodeDesc are semantically same. */ private static boolean isSame(List list1, List list2) { if (list1 != list2) { @@ -430,6 +428,7 @@ private static boolean isSame(List list1, List list2 return true; } + @SuppressWarnings("unchecked") public static T getChildOperator(Operator op, Class klazz) throws SemanticException { if (klazz.isInstance(op)) { return (T) op; diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java index ba855fc..ddc7677 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java @@ -18,11 +18,16 @@ package org.apache.hadoop.hive.ql.parse.spark; -import com.google.common.base.Preconditions; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Stack; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.common.ObjectPair; -import org.apache.hadoop.hive.ql.exec.DummyStoreOperator; import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; @@ -42,12 +47,7 @@ import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; import org.apache.hadoop.hive.ql.plan.SparkWork; -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Stack; +import com.google.common.base.Preconditions; /** * GenSparkWork separates the operator tree into spark tasks. @@ -55,8 +55,6 @@ * and break the operators into work and tasks along the way. * * Cloned from GenTezWork. - * - * TODO: need to go thru this to make it fit completely to Spark. */ public class GenSparkWork implements NodeProcessor { static final private Log LOG = LogFactory.getLog(GenSparkWork.class.getName()); @@ -84,6 +82,7 @@ public Object process(Node nd, Stack stack, "AssertionError: expected context.currentRootOperator to be not null"); // Operator is a file sink or reduce sink. Something that forces a new vertex. + @SuppressWarnings("unchecked") Operator operator = (Operator) nd; // root is the start of the operator pipeline we're currently @@ -122,7 +121,8 @@ public Object process(Node nd, Stack stack, if (smbOp != null) { // This logic is for SortMergeBucket MapJoin case. // This MapWork (of big-table, see above..) is later initialized by SparkMapJoinFactory - // processor, so don't initialize it here. Just keep track of it in the context, for later processing. + // processor, so don't initialize it here. Just keep track of it in the context, + // for later processing. work = utils.createMapWork(context, root, sparkWork, null, true); if (context.smbJoinWorkMap.get(smbOp) != null) { throw new SemanticException("Each SMBMapJoin should be associated only with one Mapwork"); @@ -182,9 +182,9 @@ public Object process(Node nd, Stack stack, work.addDummyOp((HashTableDummyOperator) dummy); } } - for (Entry parentWorkMap : linkWorkMap.entrySet()) { + for (Entry parentWorkMap : linkWorkMap.entrySet()) { BaseWork parentWork = parentWorkMap.getKey(); - LOG.debug("connecting "+parentWork.getName()+" with "+work.getName()); + LOG.debug("connecting " + parentWork.getName() + " with " + work.getName()); SparkEdgeProperty edgeProp = parentWorkMap.getValue(); sparkWork.connect(parentWork, work, edgeProp); @@ -218,7 +218,8 @@ public Object process(Node nd, Stack stack, ReduceWork reduceWork = (ReduceWork) work; for (Operator parent : new ArrayList>(root.getParentOperators())) { Preconditions.checkArgument(parent instanceof ReduceSinkOperator, - "AssertionError: expected operator to be a ReduceSinkOperator, but was " + parent.getClass().getName()); + "AssertionError: expected operator to be a ReduceSinkOperator, but was " + + parent.getClass().getName()); ReduceSinkOperator rsOp = (ReduceSinkOperator) parent; SparkEdgeProperty edgeProp = GenSparkUtils.getEdgeProperty(rsOp, reduceWork); @@ -251,7 +252,8 @@ public Object process(Node nd, Stack stack, // Also note: the concept of leaf and root is reversed in hive for historical // reasons. Roots are data sources, leaves are data sinks. I know. if (context.leafOpToFollowingWorkInfo.containsKey(operator)) { - ObjectPair childWorkInfo = context.leafOpToFollowingWorkInfo.get(operator); + ObjectPair childWorkInfo = context. + leafOpToFollowingWorkInfo.get(operator); SparkEdgeProperty edgeProp = childWorkInfo.getFirst(); ReduceWork childWork = childWorkInfo.getSecond(); @@ -286,8 +288,8 @@ public Object process(Node nd, Stack stack, // the next item will be a new root. if (!operator.getChildOperators().isEmpty()) { Preconditions.checkArgument(operator.getChildOperators().size() == 1, - "AssertionError: expected operator.getChildOperators().size() to be 1, but was " + - operator.getChildOperators().size()); + "AssertionError: expected operator.getChildOperators().size() to be 1, but was " + + operator.getChildOperators().size()); context.parentOfRoot = operator; context.currentRootOperator = operator.getChildOperators().get(0); context.preceedingWork = work; diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java index ac7b950..d0020d6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java @@ -50,6 +50,7 @@ public GenSparkWorkWalker(Dispatcher disp, GenSparkProcContext ctx) { this.ctx = ctx; } + @SuppressWarnings("unchecked") private void setRoot(Node nd) { ctx.currentRootOperator = (Operator) nd; ctx.preceedingWork = null; diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java index 836704a..c2eeb5c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java @@ -87,14 +87,12 @@ /** * SparkCompiler translates the operator plan into SparkTasks. * - * Pretty much cloned from TezCompiler. - * - * TODO: need to complete and make it fit to Spark. + * Cloned from TezCompiler. */ public class SparkCompiler extends TaskCompiler { private static final String CLASS_NAME = SparkCompiler.class.getName(); - private static final PerfLogger perfLogger = PerfLogger.getPerfLogger(); - private static final Log logger = LogFactory.getLog(SparkCompiler.class); + private static final PerfLogger PERF_LOGGER = PerfLogger.getPerfLogger(); + private static final Log LOGGER = LogFactory.getLog(SparkCompiler.class); public SparkCompiler() { } @@ -102,16 +100,12 @@ public SparkCompiler() { @Override public void init(HiveConf conf, LogHelper console, Hive db) { super.init(conf, console, db); - -// TODO: Need to check if we require the use of recursive input dirs for union processing -// conf.setBoolean("mapred.input.dir.recursive", true); -// HiveConf.setBoolVar(conf, ConfVars.HIVE_HADOOP_SUPPORTS_SUBDIRECTORIES, true); } @Override protected void optimizeOperatorPlan(ParseContext pCtx, Set inputs, Set outputs) throws SemanticException { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_OPERATOR_TREE); + PERF_LOGGER.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_OPERATOR_TREE); // Sequence of TableScan operators to be walked Deque> deque = new LinkedList>(); deque.addAll(pCtx.getTopOps().values()); @@ -137,7 +131,7 @@ protected void optimizeOperatorPlan(ParseContext pCtx, Set inputs, ArrayList topNodes = new ArrayList(); topNodes.addAll(pCtx.getTopOps().values()); ogw.startWalking(topNodes, null); - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_OPERATOR_TREE); + PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_OPERATOR_TREE); } /** @@ -147,7 +141,7 @@ protected void optimizeOperatorPlan(ParseContext pCtx, Set inputs, protected void generateTaskTree(List> rootTasks, ParseContext pCtx, List> mvTask, Set inputs, Set outputs) throws SemanticException { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_GENERATE_TASK_TREE); + PERF_LOGGER.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_GENERATE_TASK_TREE); GenSparkUtils.getUtils().resetSequenceNumber(); ParseContext tempParseContext = getParseContext(pCtx, rootTasks); @@ -244,7 +238,7 @@ public Object process(Node currNode, Stack stack, GenSparkUtils.getUtils().processFileSink(procCtx, fileSink); } - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_GENERATE_TASK_TREE); + PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_GENERATE_TASK_TREE); } @Override @@ -301,7 +295,7 @@ protected void decideExecMode(List> rootTasks, Cont @Override protected void optimizeTaskPlan(List> rootTasks, ParseContext pCtx, Context ctx) throws SemanticException { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_TASK_TREE); + PERF_LOGGER.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_TASK_TREE); PhysicalContext physicalCtx = new PhysicalContext(conf, pCtx, pCtx.getContext(), rootTasks, pCtx.getFetchTask()); @@ -345,7 +339,7 @@ protected void optimizeTaskPlan(List> rootTasks, Pa LOG.debug("Skipping stage id rearranger"); } - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_TASK_TREE); + PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_TASK_TREE); return; } } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkFileSinkProcessor.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkFileSinkProcessor.java index 365e538..7104f89 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkFileSinkProcessor.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkFileSinkProcessor.java @@ -29,11 +29,11 @@ import org.apache.hadoop.hive.ql.parse.SemanticException; /** - * FileSinkProcessor handles addition of merge, move and stats tasks for filesinks - * Cloned from tez's FileSinkProcessor + * FileSinkProcessor handles addition of merge, move and stats tasks for filesinks. + * Cloned from tez's FileSinkProcessor. */ public class SparkFileSinkProcessor implements NodeProcessor { - private static final Log logger = LogFactory.getLog(SparkFileSinkProcessor.class.getName()); + private static final Log LOGGER = LogFactory.getLog(SparkFileSinkProcessor.class.getName()); /* * (non-Javadoc) diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java index 5a68990..bc4a92c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hive.ql.parse.spark; -import java.lang.StringBuffer; import java.util.List; import java.util.Set; import java.util.Stack; @@ -57,16 +56,16 @@ * (normal, no scan, partial scan.) The plan at this point will be a single * table scan operator. * - * TODO: cloned from tez ProcessAnalyzeTable. Need to make sure it fits to Spark. + * Cloned from Tez ProcessAnalyzeTable. */ public class SparkProcessAnalyzeTable implements NodeProcessor { - private static final Log logger = LogFactory.getLog(SparkProcessAnalyzeTable.class.getName()); + private static final Log LOGGER = LogFactory.getLog(SparkProcessAnalyzeTable.class.getName()); // shared plan utils for spark private GenSparkUtils utils = null; /** - * Injecting the utils in the constructor facilitates testing + * Injecting the utils in the constructor facilitates testing. */ public SparkProcessAnalyzeTable(GenSparkUtils utils) { this.utils = utils; @@ -81,16 +80,18 @@ public Object process(Node nd, Stack stack, TableScanOperator tableScan = (TableScanOperator) nd; ParseContext parseContext = context.parseContext; + + @SuppressWarnings("rawtypes") Class inputFormat = parseContext.getTopToTable().get(tableScan) .getInputFormatClass(); QB queryBlock = parseContext.getQB(); QBParseInfo parseInfo = parseContext.getQB().getParseInfo(); if (parseInfo.isAnalyzeCommand()) { - Preconditions.checkArgument(tableScan.getChildOperators() == null || - tableScan.getChildOperators().size() == 0, - "AssertionError: expected tableScan.getChildOperators() to be null, " + - "or tableScan.getChildOperators().size() to be 0"); + Preconditions.checkArgument(tableScan.getChildOperators() == null + || tableScan.getChildOperators().size() == 0, + "AssertionError: expected tableScan.getChildOperators() to be null, " + + "or tableScan.getChildOperators().size() to be 0"); String alias = null; for (String a: parseContext.getTopOps().keySet()) { @@ -185,6 +186,8 @@ private void handlePartialScanCommand(TableScanOperator tableScan, ParseContext // partial scan task DriverContext driverCxt = new DriverContext(); + + @SuppressWarnings("unchecked") Task partialScanTask = TaskFactory.get(scanWork, parseContext.getConf()); partialScanTask.initialize(parseContext.getConf(), null, driverCxt); partialScanTask.setWork(scanWork); diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/SparkEdgeProperty.java ql/src/java/org/apache/hadoop/hive/ql/plan/SparkEdgeProperty.java index f90acf9..c38660b 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/SparkEdgeProperty.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/SparkEdgeProperty.java @@ -21,10 +21,10 @@ @Explain(displayName = "Edge Property") public class SparkEdgeProperty { - public static long SHUFFLE_NONE = 0; // No shuffle is needed. For union only. - public static long SHUFFLE_GROUP = 1; // HashPartition shuffle, keys are not sorted in any way. - public static long SHUFFLE_SORT = 2; // RangePartition shuffle, keys are total sorted. - public static long MR_SHUFFLE_SORT = 4; // HashPartition shuffle, keys are sorted by partition. + public static final long SHUFFLE_NONE = 0; // No shuffle is needed. For union only. + public static final long SHUFFLE_GROUP = 1; // HashPartition shuffle, keys are not sorted in any way. + public static final long SHUFFLE_SORT = 2; // RangePartition shuffle, keys are total sorted. + public static final long MR_SHUFFLE_SORT = 4; // HashPartition shuffle, keys are sorted by partition. private long edgeType; diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java index 99539c2..a5566b3 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java @@ -73,7 +73,7 @@ public String getName() { } /** - * getWorkMap returns a map of "vertex name" to BaseWork + * @return a map of "vertex name" to BaseWork */ @Explain(displayName = "Vertices") public Map getWorkMap() { @@ -85,7 +85,7 @@ public String getName() { } /** - * getAllWork returns a topologically sorted list of BaseWork + * @return a topologically sorted list of BaseWork */ public List getAllWork() { @@ -122,7 +122,7 @@ private void visit(BaseWork child, Set seen, List result) { } /** - * add all nodes in the collection without any connections + * Add all nodes in the collection without any connections. */ public void addAll(Collection c) { for (BaseWork w: c) { @@ -131,7 +131,7 @@ public void addAll(Collection c) { } /** - * add all nodes in the collection without any connections + * Add all nodes in the collection without any connections. */ public void addAll(BaseWork[] bws) { for (BaseWork w: bws) { @@ -260,7 +260,7 @@ public void remove(BaseWork work) { * returns the edge type connecting work a and b */ public SparkEdgeProperty getEdgeProperty(BaseWork a, BaseWork b) { - return edgeProperties.get(new ImmutablePair(a,b)); + return edgeProperties.get(new ImmutablePair(a, b)); } /** @@ -330,7 +330,9 @@ public int compareTo(Dependency o) { */ boolean dependsOn(String n1, String n2) { for (String p = dependencies.get(n1); p != null; p = dependencies.get(p)) { - if (p.equals(n2)) return true; + if (p.equals(n2)) { + return true; + } } return false; } diff --git ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorSpark.java ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorSpark.java index 03355a3..13f6024 100644 --- ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorSpark.java +++ ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorSpark.java @@ -31,9 +31,10 @@ private SparkCounters sparkCounters; + @SuppressWarnings("rawtypes") @Override public boolean connect(Configuration hconf, Task sourceTask) { - SparkTask task = (SparkTask)sourceTask; + SparkTask task = (SparkTask) sourceTask; sparkCounters = task.getSparkCounters(); if (sparkCounters == null) { return false; diff --git spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java index 0fdeff1..37607e2 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java +++ spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import com.google.common.base.Throwables; - import org.apache.hive.spark.client.metrics.Metrics; import org.apache.hive.spark.client.rpc.RpcDispatcher; import org.apache.hive.spark.counter.SparkCounters; +import com.google.common.base.Throwables; + abstract class BaseProtocol extends RpcDispatcher { protected static class CancelJob implements Serializable { @@ -118,7 +118,7 @@ } /** - * Inform the client that a new spark job has been submitted for the client job + * Inform the client that a new spark job has been submitted for the client job. */ protected static class JobSubmitted implements Serializable { final String clientJobId; diff --git spark-client/src/main/java/org/apache/hive/spark/client/Job.java spark-client/src/main/java/org/apache/hive/spark/client/Job.java index 3e2d8c9..eb07567 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/Job.java +++ spark-client/src/main/java/org/apache/hive/spark/client/Job.java @@ -17,10 +17,10 @@ package org.apache.hive.spark.client; -import java.io.Serializable; - import org.apache.hadoop.hive.common.classification.InterfaceAudience; +import java.io.Serializable; + /** * Interface for a Spark remote job. */ diff --git spark-client/src/main/java/org/apache/hive/spark/client/JobContext.java spark-client/src/main/java/org/apache/hive/spark/client/JobContext.java index 68249f9..00aa4ec 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/JobContext.java +++ spark-client/src/main/java/org/apache/hive/spark/client/JobContext.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hive.common.classification.InterfaceAudience; import org.apache.hive.spark.counter.SparkCounters; + import org.apache.spark.api.java.JavaFutureAction; import org.apache.spark.api.java.JavaSparkContext; @@ -48,7 +49,7 @@ JavaFutureAction job, SparkCounters sparkCounters, Set cachedRDDIds); /** - * Return a map from client job Id to corresponding JavaFutureActions + * Return a map from client job Id to corresponding JavaFutureActions. */ Map>> getMonitoredJobs(); diff --git spark-client/src/main/java/org/apache/hive/spark/client/JobContextImpl.java spark-client/src/main/java/org/apache/hive/spark/client/JobContextImpl.java index 6a9e8d7..1eb3ff2 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/JobContextImpl.java +++ spark-client/src/main/java/org/apache/hive/spark/client/JobContextImpl.java @@ -23,6 +23,7 @@ import java.util.concurrent.ConcurrentHashMap; import org.apache.hive.spark.counter.SparkCounters; + import org.apache.spark.api.java.JavaFutureAction; import org.apache.spark.api.java.JavaSparkContext; diff --git spark-client/src/main/java/org/apache/hive/spark/client/JobHandle.java spark-client/src/main/java/org/apache/hive/spark/client/JobHandle.java index 152899d..e760ce3 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/JobHandle.java +++ spark-client/src/main/java/org/apache/hive/spark/client/JobHandle.java @@ -22,6 +22,7 @@ import java.util.concurrent.Future; import org.apache.hadoop.hive.common.classification.InterfaceAudience; + import org.apache.hive.spark.counter.SparkCounters; /** @@ -45,12 +46,12 @@ MetricsCollection getMetrics(); /** - * Get corresponding spark job IDs for this job + * Get corresponding spark job IDs for this job. */ List getSparkJobIds(); /** - * Get the SparkCounters for this job + * Get the SparkCounters for this job. */ SparkCounters getSparkCounters(); diff --git spark-client/src/main/java/org/apache/hive/spark/client/JobHandleImpl.java spark-client/src/main/java/org/apache/hive/spark/client/JobHandleImpl.java index 7b289cf..1b8feed 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/JobHandleImpl.java +++ spark-client/src/main/java/org/apache/hive/spark/client/JobHandleImpl.java @@ -17,16 +17,16 @@ package org.apache.hive.spark.client; +import io.netty.util.concurrent.Promise; + import java.io.Serializable; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; -import io.netty.util.concurrent.Promise; - import org.apache.hive.spark.counter.SparkCounters; /** diff --git spark-client/src/main/java/org/apache/hive/spark/client/MetricsCollection.java spark-client/src/main/java/org/apache/hive/spark/client/MetricsCollection.java index 6619c88..97863f8 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/MetricsCollection.java +++ spark-client/src/main/java/org/apache/hive/spark/client/MetricsCollection.java @@ -24,13 +24,6 @@ import java.util.Set; import java.util.concurrent.locks.ReentrantReadWriteLock; -import com.google.common.base.Function; -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; -import com.google.common.collect.Collections2; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - import org.apache.hadoop.hive.common.classification.InterfaceAudience; import org.apache.hive.spark.client.metrics.DataReadMethod; import org.apache.hive.spark.client.metrics.InputMetrics; @@ -38,6 +31,13 @@ import org.apache.hive.spark.client.metrics.ShuffleReadMetrics; import org.apache.hive.spark.client.metrics.ShuffleWriteMetrics; +import com.google.common.base.Function; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import com.google.common.collect.Collections2; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + /** * Provides metrics collected for a submitted job. * @@ -162,7 +162,6 @@ private Metrics aggregate(Predicate filter) { long remoteBytesRead = 0L; // Shuffle write metrics. - boolean hasShuffleWriteMetrics = false; long shuffleBytesWritten = 0L; long shuffleWriteTime = 0L; @@ -195,7 +194,6 @@ private Metrics aggregate(Predicate filter) { } if (m.shuffleWriteMetrics != null) { - hasShuffleWriteMetrics = true; shuffleBytesWritten += m.shuffleWriteMetrics.shuffleBytesWritten; shuffleWriteTime += m.shuffleWriteMetrics.shuffleWriteTime; } diff --git spark-client/src/main/java/org/apache/hive/spark/client/MonitorCallback.java spark-client/src/main/java/org/apache/hive/spark/client/MonitorCallback.java index 744b1c3..e1e899e 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/MonitorCallback.java +++ spark-client/src/main/java/org/apache/hive/spark/client/MonitorCallback.java @@ -20,6 +20,7 @@ import java.util.Set; import org.apache.hive.spark.counter.SparkCounters; + import org.apache.spark.api.java.JavaFutureAction; interface MonitorCallback { diff --git spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java index 5aa883f..cf25268 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java +++ spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java @@ -74,7 +74,7 @@ @InterfaceAudience.Private public class RemoteDriver { - private final static Logger LOG = LoggerFactory.getLogger(RemoteDriver.class); + private static final Logger LOG = LoggerFactory.getLogger(RemoteDriver.class); private final Map> activeJobs; private final Object shutdownLock; @@ -108,8 +108,8 @@ private RemoteDriver(String[] args) throws Exception { String[] val = getArg(args, idx).split("[=]", 2); conf.set(val[0], val[1]); } else { - throw new IllegalArgumentException("Invalid command line: " + - Joiner.on(" ").join(args)); + throw new IllegalArgumentException("Invalid command line: " + + Joiner.on(" ").join(args)); } } @@ -158,7 +158,7 @@ public void rpcClosed(Rpc rpc) { } synchronized (jobQueue) { - for (Iterator> it = jobQueue.iterator(); it.hasNext(); ) { + for (Iterator> it = jobQueue.iterator(); it.hasNext();) { it.next().submit(); } } @@ -216,8 +216,8 @@ private boolean cancelJob(JobWrapper job) { private String getArg(String[] args, int keyIdx) { int valIdx = keyIdx + 1; if (args.length <= valIdx) { - throw new IllegalArgumentException("Invalid command line: " + - Joiner.on(" ").join(args)); + throw new IllegalArgumentException("Invalid command line: " + + Joiner.on(" ").join(args)); } return args[valIdx]; } @@ -382,7 +382,7 @@ public void onJobStart(SparkListenerJobStart jobStart) { public void onJobEnd(SparkListenerJobEnd jobEnd) { synchronized (stageToJobId) { for (Iterator> it = stageToJobId.entrySet().iterator(); - it.hasNext(); ) { + it.hasNext();) { Map.Entry e = it.next(); if (e.getValue() == jobEnd.jobId()) { it.remove(); @@ -398,8 +398,8 @@ public void onJobEnd(SparkListenerJobEnd jobEnd) { @Override public void onTaskEnd(SparkListenerTaskEnd taskEnd) { - if (taskEnd.reason() instanceof org.apache.spark.Success$ && - !taskEnd.taskInfo().speculative()) { + if (taskEnd.reason() instanceof org.apache.spark.Success$ + && !taskEnd.taskInfo().speculative()) { Metrics metrics = new Metrics(taskEnd.taskMetrics()); Integer jobId; synchronized (stageToJobId) { diff --git spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java index 9757ced..2abe521 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java +++ spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java @@ -17,8 +17,8 @@ package org.apache.hive.spark.client; -import java.net.URL; import java.io.Serializable; +import java.net.URL; import java.util.concurrent.Future; import org.apache.hadoop.hive.common.classification.InterfaceAudience; @@ -69,7 +69,7 @@ Future addFile(URL url); /** - * Get the count of executors + * Get the count of executors. */ Future getExecutorCount(); } diff --git spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java index 26b1d46..5e3777a 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java +++ spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java @@ -20,13 +20,12 @@ import java.io.IOException; import java.util.Map; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.Maps; -import org.apache.spark.SparkException; - import org.apache.hadoop.hive.common.classification.InterfaceAudience; import org.apache.hive.spark.client.rpc.RpcServer; +import org.apache.spark.SparkException; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; /** * Factory for SparkClient instances. diff --git spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java index 54eacfd..da3c96e 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java +++ spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java @@ -17,12 +17,16 @@ package org.apache.hive.spark.client; +import io.netty.channel.ChannelHandlerContext; +import io.netty.util.concurrent.GenericFutureListener; +import io.netty.util.concurrent.Promise; + import java.io.BufferedReader; import java.io.File; import java.io.FileOutputStream; +import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; -import java.io.IOException; import java.io.OutputStreamWriter; import java.io.Serializable; import java.io.Writer; @@ -32,33 +36,29 @@ import java.util.Properties; import java.util.UUID; import java.util.concurrent.Future; -import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hive.spark.client.rpc.Rpc; +import org.apache.hive.spark.client.rpc.RpcServer; +import org.apache.spark.SparkContext; +import org.apache.spark.SparkException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import com.google.common.base.Charsets; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import io.netty.channel.ChannelHandlerContext; -import io.netty.util.concurrent.GenericFutureListener; -import io.netty.util.concurrent.Promise; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hive.spark.client.rpc.Rpc; -import org.apache.hive.spark.client.rpc.RpcServer; -import org.apache.spark.SparkContext; -import org.apache.spark.SparkException; class SparkClientImpl implements SparkClient { private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(SparkClientImpl.class); - private static final String DEFAULT_CONNECTION_TIMEOUT = "60"; // In seconds private static final long DEFAULT_SHUTDOWN_TIMEOUT = 10000; // In milliseconds private static final String DRIVER_OPTS_KEY = "spark.driver.extraJavaOptions"; @@ -206,7 +206,7 @@ public void run() { } } String driverJavaOpts = Joiner.on(" ").skipNulls().join( - "-Dhive.spark.log.dir=" + sparkLogDir,conf.get(DRIVER_OPTS_KEY)); + "-Dhive.spark.log.dir=" + sparkLogDir, conf.get(DRIVER_OPTS_KEY)); String executorJavaOpts = Joiner.on(" ").skipNulls().join( "-Dhive.spark.log.dir=" + sparkLogDir, conf.get(EXECUTOR_OPTS_KEY)); diff --git spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java index 7e71d12..e46b67d 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java +++ spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java @@ -36,6 +36,7 @@ private InputMetrics() { // For Serialization only. this(null, 0L); } + public InputMetrics( DataReadMethod readMethod, long bytesRead) { diff --git spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java index 28a476b..a7305cf 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java +++ spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java @@ -98,15 +98,15 @@ public Metrics(TaskMetrics metrics) { optionalShuffleWriteMetrics(metrics)); } - private static final InputMetrics optionalInputMetric(TaskMetrics metrics) { + private static InputMetrics optionalInputMetric(TaskMetrics metrics) { return metrics.inputMetrics().isDefined() ? new InputMetrics(metrics) : null; } - private static final ShuffleReadMetrics optionalShuffleReadMetric(TaskMetrics metrics) { + private static ShuffleReadMetrics optionalShuffleReadMetric(TaskMetrics metrics) { return metrics.shuffleReadMetrics().isDefined() ? new ShuffleReadMetrics(metrics) : null; } - private static final ShuffleWriteMetrics optionalShuffleWriteMetrics(TaskMetrics metrics) { + private static ShuffleWriteMetrics optionalShuffleWriteMetrics(TaskMetrics metrics) { return metrics.shuffleWriteMetrics().isDefined() ? new ShuffleWriteMetrics(metrics) : null; } diff --git spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java index 5149902..ac71ae9 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java +++ spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java @@ -17,19 +17,6 @@ package org.apache.hive.spark.client.rpc; -import java.io.Closeable; -import java.util.Collection; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; @@ -37,23 +24,36 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.channel.ChannelOption; -import io.netty.channel.EventLoopGroup; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.socket.SocketChannel; import io.netty.channel.socket.nio.NioSocketChannel; -import io.netty.handler.logging.LoggingHandler; import io.netty.handler.logging.LogLevel; +import io.netty.handler.logging.LoggingHandler; import io.netty.util.concurrent.EventExecutorGroup; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.GenericFutureListener; import io.netty.util.concurrent.ImmediateEventExecutor; import io.netty.util.concurrent.Promise; import io.netty.util.concurrent.ScheduledFuture; + +import java.io.Closeable; +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.hadoop.hive.common.classification.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.common.classification.InterfaceAudience; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; /** * Encapsulates the RPC functionality. Provides higher-level methods to talk to the remote @@ -215,6 +215,7 @@ static Rpc createEmbedded(RpcDispatcher dispatcher) { private final EventExecutorGroup egroup; private final Object channelLock; + @SuppressWarnings({ "rawtypes", "unchecked" }) private Rpc(Channel channel, RpcDispatcher dispatcher, EventExecutorGroup egroup) { Preconditions.checkArgument(channel != null); Preconditions.checkArgument(dispatcher != null); @@ -331,7 +332,7 @@ public void close() { } } - public static interface Listener { + public interface Listener { void rpcClosed(Rpc rpc); diff --git spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java index 584ee48..5a826ba 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java +++ spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java @@ -38,37 +38,37 @@ private static final Logger LOG = LoggerFactory.getLogger(RpcConfiguration.class); /** Connection timeout for RPC clients. */ - public final static String CONNECT_TIMEOUT_MS_KEY = "hive.spark.client.connect.timeout.ms"; - private final static int CONNECT_TIMEOUT_MS_DEFAULT = 1000; + public static final String CONNECT_TIMEOUT_MS_KEY = "hive.spark.client.connect.timeout.ms"; + private static final int CONNECT_TIMEOUT_MS_DEFAULT = 1000; /** - * How long the server should wait for clients to connect back after they're registered. Also - * used to time out the client waiting for the server to reply to its "hello" message. + * How long the server should wait for clients to connect back after they're + * registered. Also used to time out the client waiting for the server to + * reply to its "hello" message. */ - public final static String SERVER_CONNECT_TIMEOUT_MS_KEY = - "hive.spark.client.server.connect.timeout.ms"; - private final static long SERVER_CONNECT_TIMEOUT_MS_DEFAULT = 10000L; + public static final String SERVER_CONNECT_TIMEOUT_MS_KEY = "hive.spark.client.server.connect.timeout.ms"; + private static final long SERVER_CONNECT_TIMEOUT_MS_DEFAULT = 10000L; /** - * Number of bits of randomness in the generated client secrets. Rounded down to the nearest - * multiple of 8. + * Number of bits of randomness in the generated client secrets. Rounded down + * to the nearest multiple of 8. */ - public final static String SECRET_RANDOM_BITS_KEY = "hive.spark.client.secret.bits"; - private final static int SECRET_RANDOM_BITS_DEFAULT = 256; + public static final String SECRET_RANDOM_BITS_KEY = "hive.spark.client.secret.bits"; + private static final int SECRET_RANDOM_BITS_DEFAULT = 256; /** Hostname or IP address to advertise for the server. */ - public final static String SERVER_LISTEN_ADDRESS_KEY = "hive.spark.client.server.address"; + public static final String SERVER_LISTEN_ADDRESS_KEY = "hive.spark.client.server.address"; /** Maximum number of threads to use for the RPC event loop. */ - public final static String RPC_MAX_THREADS_KEY = "hive.spark.client.rpc.threads"; - public final static int RPC_MAX_THREADS_DEFAULT = 8; + public static final String RPC_MAX_THREADS_KEY = "hive.spark.client.rpc.threads"; + public static final int RPC_MAX_THREADS_DEFAULT = 8; /** Maximum message size. Default = 10MB. */ - public final static String RPC_MAX_MESSAGE_SIZE_KEY = "hive.spark.client.rpc.max.size"; - public final static int RPC_MAX_MESSAGE_SIZE_DEFAULT = 50 * 1024 * 1024; + public static final String RPC_MAX_MESSAGE_SIZE_KEY = "hive.spark.client.rpc.max.size"; + public static final int RPC_MAX_MESSAGE_SIZE_DEFAULT = 50 * 1024 * 1024; /** Channel logging level. */ - public final static String RPC_CHANNEL_LOG_LEVEL_KEY = "hive.spark.client.channel.log.level"; + public static final String RPC_CHANNEL_LOG_LEVEL_KEY = "hive.spark.client.channel.log.level"; private final Map config; @@ -104,7 +104,8 @@ String getServerAddress() throws IOException { InetAddress address = InetAddress.getLocalHost(); if (address.isLoopbackAddress()) { - // Address resolves to something like 127.0.1.1, which happens on Debian; try to find + // Address resolves to something like 127.0.1.1, which happens on Debian; + // try to find // a better address using the local network interfaces Enumeration ifaces = NetworkInterface.getNetworkInterfaces(); while (ifaces.hasMoreElements()) { @@ -112,17 +113,13 @@ String getServerAddress() throws IOException { Enumeration addrs = ni.getInetAddresses(); while (addrs.hasMoreElements()) { InetAddress addr = addrs.nextElement(); - if (!addr.isLinkLocalAddress() && - !addr.isLoopbackAddress() && - addr instanceof Inet4Address) { + if (!addr.isLinkLocalAddress() && !addr.isLoopbackAddress() + && addr instanceof Inet4Address) { // We've found an address that looks reasonable! - LOG.warn("Your hostname, {}, resolves to a loopback address; using {} " + - " instead (on interface {})", - address.getHostName(), - addr.getHostAddress(), - ni.getName()); - LOG.warn("Set '{}' if you need to bind to another address.", - SERVER_LISTEN_ADDRESS_KEY); + LOG.warn("Your hostname, {}, resolves to a loopback address; using {} " + + " instead (on interface {})", address.getHostName(), addr.getHostAddress(), + ni.getName()); + LOG.warn("Set '{}' if you need to bind to another address.", SERVER_LISTEN_ADDRESS_KEY); return addr.getHostAddress(); } } diff --git spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcDispatcher.java spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcDispatcher.java index ab8f204..ebafd13 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcDispatcher.java +++ spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcDispatcher.java @@ -94,7 +94,7 @@ protected final void channelRead0(ChannelHandlerContext ctx, Object msg) throws } private OutstandingRpc findRpc(long id) { - for (Iterator it = rpcs.iterator(); it.hasNext(); ) { + for (Iterator it = rpcs.iterator(); it.hasNext();) { OutstandingRpc rpc = it.next(); if (rpc.id == id) { it.remove(); diff --git spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java index 3e44d96..5e18a3c 100644 --- spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java +++ spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java @@ -17,19 +17,6 @@ package org.apache.hive.spark.client.rpc; -import java.io.Closeable; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.security.SecureRandom; -import java.util.Collection; -import java.util.Iterator; -import java.util.Map; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.TimeUnit; - -import com.google.common.base.Optional; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.netty.bootstrap.ServerBootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; @@ -44,10 +31,23 @@ import io.netty.util.concurrent.Promise; import io.netty.util.concurrent.ScheduledFuture; +import java.io.Closeable; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.security.SecureRandom; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + import org.apache.hadoop.hive.common.classification.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + /** * An RPC server. The server matches remote clients based on a secret that is generated on * the server - the secret needs to be given to the client through some other mechanism for @@ -173,7 +173,7 @@ public int getPort() { public void close() { try { channel.close(); - for (Iterator clients = pendingClients.iterator(); clients.hasNext(); ) { + for (Iterator clients = pendingClients.iterator(); clients.hasNext();) { ClientInfo client = clients.next(); clients.remove(); client.promise.cancel(true); @@ -191,7 +191,7 @@ public void close() { protected void handle(ChannelHandlerContext ctx, Rpc.Hello msg) { cancelTask.cancel(true); - for (Iterator clients = pendingClients.iterator(); clients.hasNext(); ) { + for (Iterator clients = pendingClients.iterator(); clients.hasNext();) { ClientInfo client = clients.next(); if (client.secret.equals(msg.secret)) { rpc.replaceDispatcher(client.dispatcher); diff --git spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounters.java spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounters.java index 6bf3d63..5523333 100644 --- spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounters.java +++ spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounters.java @@ -23,8 +23,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.spark.api.java.JavaSparkContext; /** @@ -124,7 +122,7 @@ public String toString() { StringBuilder sb = new StringBuilder(); Map groups = getSparkCounterGroups(); if (groups != null) { - for(Map.Entry groupEntry : groups.entrySet()) { + for (Map.Entry groupEntry : groups.entrySet()) { String groupName = groupEntry.getKey(); SparkCounterGroup group = groupEntry.getValue(); sb.append(groupName).append("\n"); diff --git spark-client/src/test/java/org/apache/hive/spark/client/TestMetricsCollection.java spark-client/src/test/java/org/apache/hive/spark/client/TestMetricsCollection.java index f87fc28..5146e91 100644 --- spark-client/src/test/java/org/apache/hive/spark/client/TestMetricsCollection.java +++ spark-client/src/test/java/org/apache/hive/spark/client/TestMetricsCollection.java @@ -17,14 +17,20 @@ package org.apache.hive.spark.client; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + import java.util.Arrays; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Sets; +import org.apache.hive.spark.client.metrics.DataReadMethod; +import org.apache.hive.spark.client.metrics.InputMetrics; +import org.apache.hive.spark.client.metrics.Metrics; +import org.apache.hive.spark.client.metrics.ShuffleReadMetrics; +import org.apache.hive.spark.client.metrics.ShuffleWriteMetrics; import org.junit.Test; -import static org.junit.Assert.*; -import org.apache.hive.spark.client.metrics.*; +import com.google.common.collect.ImmutableSet; public class TestMetricsCollection { diff --git spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java index 759be05..795d62c 100644 --- spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java +++ spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java @@ -268,15 +268,6 @@ public void call(Integer l) throws Exception { } - private static class ErrorJob implements Job { - - @Override - public String call(JobContext jc) { - throw new IllegalStateException("This job does not work."); - } - - } - private static class JarJob implements Job, Function { @Override @@ -299,10 +290,6 @@ public String call(Integer i) throws Exception { private final String fileName; - FileJob() { - this(null); - } - FileJob(String fileName) { this.fileName = fileName; } @@ -346,7 +333,7 @@ public void call(Integer l) throws Exception { } - private static abstract class TestFunction { + private abstract static class TestFunction { abstract void call(SparkClient client) throws Exception; void config(Map conf) { } } diff --git spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java index bc9aa43..af53437 100644 --- spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java +++ spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java @@ -17,16 +17,21 @@ package org.apache.hive.spark.client.rpc; -import java.util.List; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; -import com.google.common.collect.Lists; import io.netty.buffer.ByteBuf; import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.logging.LoggingHandler; +import java.util.List; + import org.junit.Test; -import static org.junit.Assert.*; + +import com.google.common.collect.Lists; public class TestKryoMessageCodec { diff --git spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java index 8ae45e6..a2dd3e6 100644 --- spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java +++ spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java @@ -38,7 +38,10 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; public class TestRpc { @@ -255,6 +258,5 @@ protected void handle(ChannelHandlerContext ctx, NotDeserializable msg) { // No op. Shouldn't actually be called, if it is, the test will fail. } - }; - + } }