From f020bdd0ead06de5903a251fe02a534880420e35 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=EA=B9=80=ED=98=95=EC=A4=80?= Date: Tue, 5 Aug 2014 20:26:38 +0900 Subject: [PATCH 1/8] TAJO-992: Reduce number of hash shuffle output file. --- .../java/org/apache/tajo/conf/TajoConf.java | 5 +- .../engine/planner/PhysicalPlannerImpl.java | 1 + .../planner/physical/ExternalSortExec.java | 3 + .../physical/HashShuffleFileWriteExec.java | 154 +++++++++----- .../org/apache/tajo/master/GlobalEngine.java | 2 +- .../tajo/master/querymaster/QueryUnit.java | 4 + .../master/querymaster/Repartitioner.java | 190 +++++++++++++----- .../tajo/master/querymaster/SubQuery.java | 21 +- .../java/org/apache/tajo/worker/Fetcher.java | 64 ++++-- .../org/apache/tajo/worker/TajoWorker.java | 14 ++ .../java/org/apache/tajo/worker/Task.java | 27 +-- .../tajo/worker/TaskAttemptContext.java | 43 +++- .../org/apache/tajo/worker/TaskRunner.java | 9 +- .../apache/tajo/worker/TaskRunnerManager.java | 28 ++- .../planner/physical/TestPhysicalPlanner.java | 34 ++-- .../tajo/engine/query/TestJoinQuery.java | 6 +- .../TestQueryUnitStatusUpdate.java | 6 +- .../tajo/storage/AbstractStorageManager.java | 1 - .../tajo/storage/HashShuffleAppender.java | 99 +++++++++ .../storage/HashShuffleAppenderManager.java | 169 ++++++++++++++++ .../java/org/apache/tajo/storage/RawFile.java | 1 + .../pullserver/TajoPullServerService.java | 47 +++-- 22 files changed, 730 insertions(+), 198 deletions(-) create mode 100644 tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java create mode 100644 tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index d5e8bc41d9..9f13b9e484 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -250,6 +250,7 @@ public static enum ConfVars { DIST_QUERY_GROUPBY_PARTITION_VOLUME("tajo.dist-query.groupby.partition-volume-mb", 256), DIST_QUERY_TABLE_PARTITION_VOLUME("tajo.dist-query.table-partition.task-volume-mb", 256), + DIST_QUERY_CLUSTER_SLOT_MAX_RATIO("tajo.dist-query.cluster-slot.max-ratio", 1.0f), ////////////////////////////////// // Physical Executors @@ -347,7 +348,9 @@ public static enum ConfVars { TESTCASE_MIN_TASK_NUM("tajo.testcase.min.task.num", -1), // behavior control - BEHAVIOR_ARITHMETIC_ABORT("tajo.behavior.arithmetic-abort", false); + BEHAVIOR_ARITHMETIC_ABORT("tajo.behavior.arithmetic-abort", false), + + HASH_SHUFFLE_PARENT_DIRS("tajo.hash.shuffle.parent.dirs.count", 10); ; public final String varname; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java index 6678e469ae..664c4670ea 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java @@ -975,6 +975,7 @@ private PhysicalExec createBestAggregationPlan(TaskAttemptContext context, Group // if the relation size is less than the threshold, // the hash aggregation will be used. + LOG.info("Aggregation:estimatedSize=" + estimatedSize + ", threshold=" + threshold); if (estimatedSize <= threshold) { LOG.info("The planner chooses [Hash Aggregation]"); return createInMemoryHashAggregation(context, groupbyNode, subOp); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java index f71475867c..85634db179 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java @@ -701,6 +701,9 @@ public TableStats getInputStats() { return mergerInputStats; } TableStats leftInputStats = leftScan.getInputStats(); + if (mergerInputStats == null) { + mergerInputStats = new TableStats(); + } mergerInputStats.setNumBytes(0); mergerInputStats.setReadBytes(0); mergerInputStats.setNumRows(0); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java index 44e8646963..2ac36fe9df 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java @@ -21,17 +21,16 @@ import com.google.common.base.Preconditions; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.Column; import org.apache.tajo.catalog.TableMeta; import org.apache.tajo.catalog.statistics.StatisticsUtil; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.engine.planner.logical.ShuffleFileWriteNode; -import org.apache.tajo.storage.*; +import org.apache.tajo.storage.AbstractStorageManager; +import org.apache.tajo.storage.HashShuffleAppender; +import org.apache.tajo.storage.HashShuffleAppenderManager; +import org.apache.tajo.storage.Tuple; import org.apache.tajo.worker.TaskAttemptContext; import java.io.IOException; @@ -49,11 +48,12 @@ public final class HashShuffleFileWriteExec extends UnaryPhysicalExec { private ShuffleFileWriteNode plan; private final TableMeta meta; private Partitioner partitioner; - private final Path storeTablePath; - private Map appenderMap = new HashMap(); +// private final Path storeTablePath; + private Map appenderMap = new HashMap(); private final int numShuffleOutputs; private final int [] shuffleKeyIds; - + private HashShuffleAppenderManager hashShuffleAppenderManager; + public HashShuffleFileWriteExec(TaskAttemptContext context, final AbstractStorageManager sm, final ShuffleFileWriteNode plan, final PhysicalExec child) throws IOException { super(context, plan.getInSchema(), plan.getOutSchema(), child); @@ -73,71 +73,115 @@ public HashShuffleFileWriteExec(TaskAttemptContext context, final AbstractStorag i++; } this.partitioner = new HashPartitioner(shuffleKeyIds, numShuffleOutputs); - storeTablePath = new Path(context.getWorkDir(), "output"); + this.hashShuffleAppenderManager = context.getHashShuffleAppenderManager(); } @Override public void init() throws IOException { super.init(); - FileSystem fs = new RawLocalFileSystem(); - fs.mkdirs(storeTablePath); } - private Appender getAppender(int partId) throws IOException { - Appender appender = appenderMap.get(partId); - + private HashShuffleAppender getAppender(int partId) throws IOException { + HashShuffleAppender appender = appenderMap.get(partId); if (appender == null) { - Path dataFile = getDataFile(partId); - FileSystem fs = dataFile.getFileSystem(context.getConf()); - if (fs.exists(dataFile)) { - LOG.info("File " + dataFile + " already exists!"); - FileStatus status = fs.getFileStatus(dataFile); - LOG.info("File size: " + status.getLen()); - } - appender = StorageManagerFactory.getStorageManager(context.getConf()).getAppender(meta, outSchema, dataFile); - appender.enableStats(); - appender.init(); + appender = hashShuffleAppenderManager.getAppender(context.getConf(), + context.getQueryId().getQueryUnitId().getExecutionBlockId(), partId, meta, outSchema); appenderMap.put(partId, appender); - } else { - appender = appenderMap.get(partId); } - return appender; } - private Path getDataFile(int partId) { - return StorageUtil.concatPath(storeTablePath, ""+partId); - } + Map partitionStats = new HashMap(); + Map> partitionTuples = new HashMap>(); @Override public Tuple next() throws IOException { - Tuple tuple; - Appender appender; - int partId; - while ((tuple = child.next()) != null) { - partId = partitioner.getPartition(tuple); - appender = getAppender(partId); - appender.addTuple(tuple); - } - - List statSet = new ArrayList(); - for (Map.Entry entry : appenderMap.entrySet()) { - int partNum = entry.getKey(); - Appender app = entry.getValue(); - app.flush(); - app.close(); - statSet.add(app.getStats()); - if (app.getStats().getNumRows() > 0) { - context.addShuffleFileOutput(partNum, getDataFile(partNum).getName()); - context.addPartitionOutputVolume(partNum, app.getStats().getNumBytes()); + try { + Tuple tuple; + int partId; + int tupleCount = 0; + long numRows = 0; + while ((tuple = child.next()) != null) { + tupleCount++; + numRows++; + + partId = partitioner.getPartition(tuple); + List partitionTupleList = partitionTuples.get(partId); + if (partitionTupleList == null) { + partitionTupleList = new ArrayList(1000); + partitionTuples.put(partId, partitionTupleList); + } + try { + partitionTupleList.add(tuple.clone()); + } catch (CloneNotSupportedException e) { + } + if (tupleCount >= 10000) { + for (Map.Entry> entry : partitionTuples.entrySet()) { + int appendPartId = entry.getKey(); + HashShuffleAppender appender = getAppender(appendPartId); + long appendedSize = appender.addTuples(entry.getValue()); + Long previousSize = partitionStats.get(appendPartId); + if (previousSize == null) { + partitionStats.put(appendPartId, appendedSize); + } else { + partitionStats.put(appendPartId, appendedSize + previousSize); + } + entry.getValue().clear(); + } + tupleCount = 0; + } } + + // processing remained tuples + for (Map.Entry> entry : partitionTuples.entrySet()) { + int appendPartId = entry.getKey(); + HashShuffleAppender appender = getAppender(appendPartId); + long appendedSize = appender.addTuples(entry.getValue()); + Long previousSize = partitionStats.get(appendPartId); + if (previousSize == null) { + partitionStats.put(appendPartId, appendedSize); + } else { + partitionStats.put(appendPartId, appendedSize + previousSize); + } + entry.getValue().clear(); + } + + // set table stats + List statSet = new ArrayList(); + for (Integer eachPartId : partitionStats.keySet()) { + HashShuffleAppender appender = appenderMap.get(eachPartId); + TableStats appenderStat = appender.getStats(); + TableStats tableStats = null; + try { + tableStats = (TableStats) appenderStat.clone(); + } catch (CloneNotSupportedException e) { + LOG.error(e); + } + tableStats.setNumBytes(partitionStats.get(eachPartId)); + tableStats.setReadBytes(partitionStats.get(eachPartId)); + + if (numRows > 0) { + context.addShuffleFileOutput(eachPartId, + hashShuffleAppenderManager.getPartitionAppenderDataFile( + context.getQueryId().getQueryUnitId().getExecutionBlockId(), eachPartId).getName()); + context.addPartitionOutputVolume(eachPartId, tableStats.getNumBytes()); + } + statSet.add(tableStats); + } + + // Collect and aggregated statistics data + TableStats aggregated = StatisticsUtil.aggregateTableStat(statSet); + aggregated.setNumRows(numRows); + context.setResultStats(aggregated); + + partitionStats.clear(); + partitionTuples.clear(); + + return null; + } catch (Exception e) { + LOG.error(e.getMessage(), e); + throw new IOException(e); } - - // Collect and aggregated statistics data - TableStats aggregated = StatisticsUtil.aggregateTableStat(statSet); - context.setResultStats(aggregated); - - return null; } @Override diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java index 73f3cf5854..ea7bdd26dd 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java @@ -310,7 +310,7 @@ private void insertNonFromQuery(QueryContext queryContext, InsertNode insertNode } TaskAttemptContext taskAttemptContext = - new TaskAttemptContext(context.getConf(), queryContext, null, (CatalogProtos.FragmentProto[]) null, stagingDir); + new TaskAttemptContext(context.getConf(), queryContext, null, null, (CatalogProtos.FragmentProto[]) null, stagingDir); taskAttemptContext.setOutputPath(new Path(stagingResultDir, "part-01-000000")); EvalExprExec evalExprExec = new EvalExprExec(taskAttemptContext, (EvalExprNode) insertNode.getChild()); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java index 8c953bdef8..9d308350f4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java @@ -720,6 +720,10 @@ public long getVolume() { return this.volume; } + public long setVolume(long volume) { + return this.volume = volume; + } + @Override public int hashCode() { return Objects.hashCode(ebId, taskId, partId, attemptId, host); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java index f86106fa8c..28219386fc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java @@ -59,9 +59,7 @@ import java.util.*; import java.util.Map.Entry; -import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleType.HASH_SHUFFLE; -import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleType.RANGE_SHUFFLE; -import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleType.SCATTERED_HASH_SHUFFLE; +import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleType.*; /** * Repartitioner creates non-leaf tasks and shuffles intermediate data. @@ -308,7 +306,7 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster MasterPlan masterPlan = subQuery.getMasterPlan(); ExecutionBlock execBlock = subQuery.getBlock(); // The hash map is modeling as follows: - // > + // >> Map>> hashEntries = new HashMap>>(); @@ -362,12 +360,15 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster } } + // merge intermediate entry by ebid, pullhost + Map>> mergedHashEntries = mergeIntermediateByPullHost(hashEntries); + // hashEntries can be zero if there are no input data. // In the case, it will cause the zero divided exception. // it avoids this problem. int[] avgSize = new int[2]; - avgSize[0] = hashEntries.size() == 0 ? 0 : (int) (stats[0] / hashEntries.size()); - avgSize[1] = hashEntries.size() == 0 ? 0 : (int) (stats[1] / hashEntries.size()); + avgSize[0] = mergedHashEntries.size() == 0 ? 0 : (int) (stats[0] / mergedHashEntries.size()); + avgSize[1] = mergedHashEntries.size() == 0 ? 0 : (int) (stats[1] / mergedHashEntries.size()); int bothFetchSize = avgSize[0] + avgSize[1]; // Getting the desire number of join tasks according to the volumn @@ -382,10 +383,10 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster // determine the number of task per 64MB int maxTaskNum = (int) Math.ceil((double) mb / desireJoinTaskVolumn); LOG.info("The calculated number of tasks is " + maxTaskNum); - LOG.info("The number of total shuffle keys is " + hashEntries.size()); + LOG.info("The number of total shuffle keys is " + mergedHashEntries.size()); // the number of join tasks cannot be larger than the number of // distinct partition ids. - int joinTaskNum = Math.min(maxTaskNum, hashEntries.size()); + int joinTaskNum = Math.min(maxTaskNum, mergedHashEntries.size()); LOG.info("The determined number of join tasks is " + joinTaskNum); FileFragment[] rightFragments = new FileFragment[1 + (broadcastFragments == null ? 0 : broadcastFragments.length)]; @@ -397,7 +398,7 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster // Assign partitions to tasks in a round robin manner. for (Entry>> entry - : hashEntries.entrySet()) { + : mergedHashEntries.entrySet()) { addJoinShuffle(subQuery, entry.getKey(), entry.getValue()); } @@ -405,6 +406,53 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster schedulerContext.setEstimatedTaskNum(joinTaskNum); } + /** + * merge intermediate entry by ebid, pullhost + * @param hashEntries + * @return + */ + private static Map>> mergeIntermediateByPullHost( + Map>> hashEntries) { + Map>> mergedHashEntries = + new HashMap>>(); + + for(Entry>> entry: hashEntries.entrySet()) { + Integer partId = entry.getKey(); + for (Entry> partEntry: entry.getValue().entrySet()) { + List intermediateList = partEntry.getValue(); + if (intermediateList == null || intermediateList.isEmpty()) { + continue; + } + ExecutionBlockId ebId = partEntry.getKey(); + // EBID + PullHost -> IntermediateEntry + // In the case of union partEntry.getKey() return's delegated EBID. + // Intermediate entries are merged by real EBID. + Map ebMerged = new HashMap(); + + for (IntermediateEntry eachIntermediate: intermediateList) { + String ebMergedKey = eachIntermediate.getEbId().toString() + eachIntermediate.getPullHost().getPullAddress(); + IntermediateEntry intermediateEntryPerPullHost = ebMerged.get(ebMergedKey); + if (intermediateEntryPerPullHost == null) { + intermediateEntryPerPullHost = new IntermediateEntry(-1, -1, partId, eachIntermediate.getPullHost()); + intermediateEntryPerPullHost.setEbId(eachIntermediate.getEbId()); + ebMerged.put(ebMergedKey, intermediateEntryPerPullHost); + } + intermediateEntryPerPullHost.setVolume(intermediateEntryPerPullHost.getVolume() + eachIntermediate.getVolume()); + } + + List ebIntermediateEntries = new ArrayList(ebMerged.values()); + + Map> mergedPartEntries = mergedHashEntries.get(partId); + if (mergedPartEntries == null) { + mergedPartEntries = new HashMap>(); + mergedHashEntries.put(partId, mergedPartEntries); + } + mergedPartEntries.put(ebId, ebIntermediateEntries); + } + } + return mergedHashEntries; + } + /** * It creates a number of fragments for all partitions. */ @@ -694,21 +742,47 @@ public static void scheduleHashShuffledFetches(TaskSchedulerContext schedulerCon List>(); for (ExecutionBlock block : masterPlan.getChilds(execBlock)) { - List partitions = new ArrayList(); + Map> intermediatesByPartAndHost = new HashMap>(); for (QueryUnit tasks : subQuery.getContext().getSubQuery(block.getId()).getQueryUnits()) { if (tasks.getIntermediateData() != null) { - partitions.addAll(tasks.getIntermediateData()); - - // In scattered hash shuffle, Collecting each IntermediateEntry - if (channel.getShuffleType() == SCATTERED_HASH_SHUFFLE) { - if (intermediates.containsKey(block.getId())) { - intermediates.get(block.getId()).addAll(tasks.getIntermediateData()); - } else { - intermediates.put(block.getId(), tasks.getIntermediateData()); + for (IntermediateEntry eachEntry: tasks.getIntermediateData()) { + int partId = eachEntry.getPartId(); + String pullHost = eachEntry.getPullHost().getPullAddress(); + List intermediateList = intermediatesByPartAndHost.get(partId + pullHost); + if (intermediateList == null) { + intermediateList = new ArrayList(); + intermediatesByPartAndHost.put(partId + pullHost, intermediateList); } + intermediateList.add(eachEntry); } } } + // merge intermediate data by partId, pullHost + List partitions = new ArrayList(); + for (List eachList: intermediatesByPartAndHost.values()) { + if (eachList.isEmpty()) { + continue; + } + IntermediateEntry first = eachList.get(0); + IntermediateEntry mergedEntry = new IntermediateEntry(-1, -1, first.getPartId(), first.getPullHost()); + mergedEntry.setEbId(first.getEbId()); + long volume = 0; + for (IntermediateEntry eachEntry: eachList) { + volume += eachEntry.getVolume(); + } + mergedEntry.setVolume(volume); + partitions.add(mergedEntry); + } + + // In scattered hash shuffle, Collecting each IntermediateEntry + if (channel.getShuffleType() == SCATTERED_HASH_SHUFFLE) { + if (intermediates.containsKey(block.getId())) { + intermediates.get(block.getId()).addAll(partitions); + } else { + intermediates.put(block.getId(), partitions); + } + } + Map> hashed = hashByKey(partitions); for (Entry> interm : hashed.entrySet()) { hashedByHost = hashByHost(interm.getValue()); @@ -863,46 +937,54 @@ public static List createFetchURL(FetchImpl fetch, boolean includeParts) { } List fetchURLs = new ArrayList(); - if(includeParts){ - // If the get request is longer than 2000 characters, - // the long request uri may cause HTTP Status Code - 414 Request-URI Too Long. - // Refer to http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html#sec10.4.15 - // The below code transforms a long request to multiple requests. - List taskIdsParams = new ArrayList(); - StringBuilder taskIdListBuilder = new StringBuilder(); - List taskIds = fetch.getTaskIds(); - List attemptIds = fetch.getAttemptIds(); - boolean first = true; - - for (int i = 0; i < taskIds.size(); i++) { - StringBuilder taskAttemptId = new StringBuilder(); - - if (!first) { // when comma is added? - taskAttemptId.append(","); - } else { - first = false; - } + if(includeParts) { + if (fetch.getType() == HASH_SHUFFLE || fetch.getType() == SCATTERED_HASH_SHUFFLE) { + fetchURLs.add(URI.create(urlPrefix.toString())); + } else { + // If the get request is longer than 2000 characters, + // the long request uri may cause HTTP Status Code - 414 Request-URI Too Long. + // Refer to http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html#sec10.4.15 + // The below code transforms a long request to multiple requests. + List taskIdsParams = new ArrayList(); + StringBuilder taskIdListBuilder = new StringBuilder(); + List taskIds = fetch.getTaskIds(); + List attemptIds = fetch.getAttemptIds(); + boolean first = true; + + for (int i = 0; i < taskIds.size(); i++) { + StringBuilder taskAttemptId = new StringBuilder(); + + if (!first) { // when comma is added? + taskAttemptId.append(","); + } else { + first = false; + } - int taskId = taskIds.get(i); - int attemptId = attemptIds.get(i); - taskAttemptId.append(taskId).append("_").append(attemptId); + int taskId = taskIds.get(i); + if (taskId < 0) { + // In the case of hash shuffle each partition has single shuffle file per worker. + // TODO If file is large, consider multiple fetching(shuffle file can be split) + continue; + } + int attemptId = attemptIds.get(i); + taskAttemptId.append(taskId).append("_").append(attemptId); - if (taskIdListBuilder.length() + taskAttemptId.length() - > HTTP_REQUEST_MAXIMUM_LENGTH) { + if (taskIdListBuilder.length() + taskAttemptId.length() + > HTTP_REQUEST_MAXIMUM_LENGTH) { + taskIdsParams.add(taskIdListBuilder.toString()); + taskIdListBuilder = new StringBuilder(taskId + "_" + attemptId); + } else { + taskIdListBuilder.append(taskAttemptId); + } + } + // if the url params remain + if (taskIdListBuilder.length() > 0) { taskIdsParams.add(taskIdListBuilder.toString()); - taskIdListBuilder = new StringBuilder(taskId + "_" + attemptId); - } else { - taskIdListBuilder.append(taskAttemptId); } - } - // if the url params remain - if (taskIdListBuilder.length() > 0) { - taskIdsParams.add(taskIdListBuilder.toString()); - } - - urlPrefix.append("&ta="); - for (String param : taskIdsParams) { - fetchURLs.add(URI.create(urlPrefix + param)); + urlPrefix.append("&ta="); + for (String param : taskIdsParams) { + fetchURLs.add(URI.create(urlPrefix + param)); + } } } else { fetchURLs.add(URI.create(urlPrefix.toString())); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java index f2e9dd56ad..fbbd20f67e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java @@ -800,7 +800,11 @@ public static int calculateShuffleOutputNum(SubQuery subQuery, DataChannel chann LOG.info(subQuery.getId() + ", Total memory of cluster is " + totalMem + " MB"); int slots = Math.max(totalMem / conf.getIntVar(ConfVars.TASK_DEFAULT_MEMORY), 1); - int taskNum = Math.min(taskNumBySize, slots); //Maximum partitions + int maxSlots = (int)(slots * conf.getFloatVar(ConfVars.DIST_QUERY_CLUSTER_SLOT_MAX_RATIO)); + + // determine the number of task + int taskNum = Math.min(taskNumBySize, maxSlots); + LOG.info(subQuery.getId() + ", The determined number of aggregation partitions is " + taskNum); return taskNum; } @@ -1054,25 +1058,10 @@ public void transition(SubQuery subQuery, subQuery.completedTaskCount++; if (taskEvent.getState() == TaskState.SUCCEEDED) { -// if (task.isLeafTask()) { -// subQuery.succeededObjectCount += task.getTotalFragmentNum(); -// } else { -// subQuery.succeededObjectCount++; -// } subQuery.succeededObjectCount++; } else if (task.getState() == TaskState.KILLED) { -// if (task.isLeafTask()) { -// subQuery.killedObjectCount += task.getTotalFragmentNum(); -// } else { -// subQuery.killedObjectCount++; -// } subQuery.killedObjectCount++; } else if (task.getState() == TaskState.FAILED) { -// if (task.isLeafTask()) { -// subQuery.failedObjectCount+= task.getTotalFragmentNum(); -// } else { -// subQuery.failedObjectCount++; -// } subQuery.failedObjectCount++; // if at least one task is failed, try to kill all tasks. subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_KILL)); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java index 2aa28751df..8504ba1f99 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java @@ -155,6 +155,43 @@ public File get() throws IOException { timer.stop(); } } + +// URL url = new URL(uri.toString()); +// HttpURLConnection conn = (HttpURLConnection) url.openConnection(); +// conn.connect(); +// int length = conn.getContentLength(); +// byte[] byteArr = new byte[1024 * 1024]; +// InputStream in = null; +// BufferedOutputStream out = null; +// try { +// out = new BufferedOutputStream(new FileOutputStream(file)); +// in = conn.getInputStream(); +// int readBytes = 0; +// int totalReadBytes = 0; +// while ((readBytes = in.read(byteArr)) > 0) { +// out.write(byteArr, 0, readBytes); +// totalReadBytes += readBytes; +// if (totalReadBytes >= length) { +// break; +// } +// } +// state = TajoProtos.FetcherState.FETCH_FINISHED; +// } catch (Exception e) { +// state = FetcherState.FETCH_FAILED; +// } finally { +// if (in != null) { +// in.close(); +// } +// if (conn != null) { +// conn.disconnect(); +// } +// if (out != null) { +// out.close(); +// } +// this.finishTime = System.currentTimeMillis(); +// LOG.fatal("Fetcher finished: " + (this.finishTime - this.startTime) + " ms, len=" + length); +// } +// return file; } public URI getURI() { @@ -175,7 +212,6 @@ public HttpClientHandler(File file) throws FileNotFoundException { @Override public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception { - messageReceiveCount++; try { if (!readingChunks && e.getMessage() instanceof HttpResponse) { @@ -229,16 +265,8 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) HttpChunk chunk = (HttpChunk) e.getMessage(); if (chunk.isLast()) { readingChunks = false; - long fileLength = file.length(); - if (fileLength == length) { - LOG.info("Data fetch is done (total received bytes: " + fileLength - + ")"); - } else { - LOG.info("Data fetch is done, but cannot get all data " - + "(received/total: " + fileLength + "/" + length + ")"); - } } else { - if(fc != null){ + if(fc != null) { fc.write(chunk.getContent().toByteBuffer()); } } @@ -247,11 +275,17 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) if(raf != null) { fileLen = file.length(); } - - if(fileLen == length){ - IOUtils.cleanup(LOG, fc, raf); - finishTime = System.currentTimeMillis(); - state = TajoProtos.FetcherState.FETCH_FINISHED; + if (!readingChunks) { + if (fileLen == length) { + LOG.info("Data fetch is done (total received bytes: " + fileLen + + ", number of received message: " + messageReceiveCount + "), file=" + file); + IOUtils.cleanup(LOG, fc, raf); + finishTime = System.currentTimeMillis(); + state = TajoProtos.FetcherState.FETCH_FINISHED; + } else { + LOG.info("Data fetch is done, but cannot get all data " + + "(received/total: " + fileLen + "/" + length + ")"); + } } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index ed78e492fb..deb2885d3b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -42,6 +42,7 @@ import org.apache.tajo.rpc.RpcChannelFactory; import org.apache.tajo.rpc.RpcConnectionPool; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; +import org.apache.tajo.storage.HashShuffleAppenderManager; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.NetUtils; import org.apache.tajo.util.StringUtils; @@ -121,6 +122,8 @@ public class TajoWorker extends CompositeService { private TajoSystemMetrics workerSystemMetrics; + private HashShuffleAppenderManager hashShuffleAppenderManager; + public TajoWorker() throws Exception { super(TajoWorker.class.getName()); } @@ -256,6 +259,13 @@ public void init(Configuration conf) { workerHeartbeatThread = new WorkerHeartbeatService(workerContext); workerHeartbeatThread.init(conf); addIfService(workerHeartbeatThread); + + try { + hashShuffleAppenderManager = new HashShuffleAppenderManager(systemConf); + } catch (IOException e) { + LOG.fatal(e.getMessage(), e); + System.exit(-1); + } } private void initWorkerMetrics() { @@ -471,6 +481,10 @@ public boolean isTaskRunnerMode() { public TajoSystemMetrics getWorkerSystemMetrics() { return workerSystemMetrics; } + + public HashShuffleAppenderManager getHashShuffleAppenderManager() { + return hashShuffleAppenderManager; + } } public void stopWorkerForce() { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index 230c63aee8..9b2a86f35d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -133,22 +133,22 @@ public NumberFormat initialValue() { }; public Task(QueryUnitAttemptId taskId, - final TaskRunner.TaskRunnerContext worker, + final TaskRunner.TaskRunnerContext runnerContext, final QueryMasterProtocolService.Interface masterProxy, final QueryUnitRequest request) throws IOException { this.request = request; this.taskId = taskId; - this.systemConf = worker.getConf(); + this.systemConf = runnerContext.getConf(); this.queryContext = request.getQueryContext(); - this.taskRunnerContext = worker; + this.taskRunnerContext = runnerContext; this.masterProxy = masterProxy; - this.localFS = worker.getLocalFS(); - this.lDirAllocator = worker.getLocalDirAllocator(); + this.localFS = runnerContext.getLocalFS(); + this.lDirAllocator = runnerContext.getLocalDirAllocator(); this.taskDir = StorageUtil.concatPath(taskRunnerContext.getBaseDir(), taskId.getQueryUnitId().getId() + "_" + taskId.getId()); - this.context = new TaskAttemptContext(systemConf, queryContext, taskId, + this.context = new TaskAttemptContext(systemConf, queryContext, runnerContext.getWorkerContext(), taskId, request.getFragments().toArray(new FragmentProto[request.getFragments().size()]), taskDir); this.context.setDataChannel(request.getDataChannel()); this.context.setEnforcer(request.getEnforcer()); @@ -200,12 +200,14 @@ public Task(QueryUnitAttemptId taskId, LOG.info("=================================="); LOG.info("* Subquery " + request.getId() + " is initialized"); LOG.info("* InterQuery: " + interQuery - + (interQuery ? ", Use " + this.shuffleType + " shuffle":"")); + + (interQuery ? ", Use " + this.shuffleType + " shuffle":"") + + ", Fragments (num: " + request.getFragments().size() + ")" + + ", Fetches (total:" + request.getFetches().size() + ") :"); - LOG.info("* Fragments (num: " + request.getFragments().size() + ")"); - LOG.info("* Fetches (total:" + request.getFetches().size() + ") :"); - for (FetchImpl f : request.getFetches()) { - LOG.info("Table Id: " + f.getName() + ", Simple URIs: " + f.getSimpleURIs()); + if(LOG.isDebugEnabled()) { + for (FetchImpl f : request.getFetches()) { + LOG.debug("Table Id: " + f.getName() + ", Simple URIs: " + f.getSimpleURIs()); + } } LOG.info("* Local task dir: " + taskDir); if(LOG.isDebugEnabled()) { @@ -485,7 +487,8 @@ public void run() { } finishTime = System.currentTimeMillis(); - LOG.info("Worker's task counter - total:" + taskRunnerContext.completedTasksNum.intValue() + + LOG.info(context.getTaskId() + " completed. " + + "Worker's task counter - total:" + taskRunnerContext.completedTasksNum.intValue() + ", succeeded: " + taskRunnerContext.succeededTasksNum.intValue() + ", killed: " + taskRunnerContext.killedTasksNum.intValue() + ", failed: " + taskRunnerContext.failedTasksNum.intValue()); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java index db4af458ea..6103afe74a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java @@ -31,12 +31,15 @@ import org.apache.tajo.engine.planner.enforce.Enforcer; import org.apache.tajo.engine.planner.global.DataChannel; import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.storage.HashShuffleAppenderManager; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.storage.fragment.Fragment; import org.apache.tajo.storage.fragment.FragmentConvertor; import org.apache.tajo.util.TUtil; +import org.apache.tajo.worker.TajoWorker.WorkerContext; import java.io.File; +import java.io.IOException; import java.util.*; import java.util.Map.Entry; import java.util.concurrent.CountDownLatch; @@ -72,15 +75,21 @@ public class TaskAttemptContext { private DataChannel dataChannel; private Enforcer enforcer; private QueryContext queryContext; + private WorkerContext workerContext; /** a output volume for each partition */ private Map partitionOutputVolume; + private HashShuffleAppenderManager hashShuffleAppenderManager; - public TaskAttemptContext(TajoConf conf, QueryContext queryContext, final QueryUnitAttemptId queryId, + public TaskAttemptContext(final TajoConf conf, + final QueryContext queryContext, + final WorkerContext workerContext, + final QueryUnitAttemptId queryId, final FragmentProto[] fragments, final Path workDir) { this.conf = conf; this.queryContext = queryContext; + this.workerContext = workerContext; this.queryId = queryId; if (fragments != null) { @@ -101,12 +110,26 @@ public TaskAttemptContext(TajoConf conf, QueryContext queryContext, final QueryU state = TaskAttemptState.TA_PENDING; this.partitionOutputVolume = Maps.newHashMap(); + + if (workerContext != null) { + this.hashShuffleAppenderManager = workerContext.getHashShuffleAppenderManager(); + } else { + LOG.error("HashShuffleAppenderManager "); + try { + this.hashShuffleAppenderManager = new HashShuffleAppenderManager(conf); + } catch (IOException e) { + LOG.error(e.getMessage(), e); + } + } } @VisibleForTesting - public TaskAttemptContext(TajoConf conf, QueryContext queryContext, final QueryUnitAttemptId queryId, - final Fragment [] fragments, final Path workDir) { - this(conf, queryContext, queryId, FragmentConvertor.toFragmentProtoArray(fragments), workDir); + public TaskAttemptContext(final TajoConf conf, + final QueryContext queryContext, + final QueryUnitAttemptId queryId, + final Fragment [] fragments, + final Path workDir) { + this(conf, queryContext, null, queryId, FragmentConvertor.toFragmentProtoArray(fragments), workDir); } public TajoConf getConf() { @@ -332,4 +355,16 @@ public boolean equals(Object obj) { public QueryContext getQueryContext() { return queryContext; } + + public WorkerContext getWorkContext() { + return workerContext; + } + + public QueryUnitAttemptId getQueryId() { + return queryId; + } + + public HashShuffleAppenderManager getHashShuffleAppenderManager() { + return hashShuffleAppenderManager; + } } \ No newline at end of file diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java index 3fcee06ab9..0e0c01dcdc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java @@ -46,6 +46,7 @@ import org.apache.tajo.rpc.NullCallback; import org.apache.tajo.rpc.RpcConnectionPool; import org.apache.tajo.util.TajoIdUtils; +import org.apache.tajo.worker.TajoWorker.WorkerContext; import java.net.InetSocketAddress; import java.util.Map; @@ -305,6 +306,10 @@ public void addTaskHistory(QueryUnitAttemptId quAttemptId, TaskHistory taskHisto public TaskRunnerHistory getExcutionBlockHistory(){ return history; } + + public WorkerContext getWorkerContext() { + return taskRunnerManager.getWorkerContext(); + } } public TaskRunnerContext getContext() { @@ -364,7 +369,9 @@ public void run() { } // if there has been no assigning task for a given period, // TaskRunner will retry to request an assigning task. - LOG.info("Retry assigning task:" + getId()); + if (LOG.isDebugEnabled()) { + LOG.info("Retry assigning task:" + getId()); + } continue; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java index a8e8730d4a..6e5c4fd144 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java @@ -23,11 +23,13 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.CompositeService; +import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.QueryUnitAttemptId; import org.apache.tajo.conf.TajoConf; import java.util.*; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; public class TaskRunnerManager extends CompositeService { private static final Log LOG = LogFactory.getLog(TaskRunnerManager.class); @@ -88,7 +90,20 @@ public void stop() { public void stopTask(String id) { LOG.info("Stop Task:" + id); synchronized(taskRunnerMap) { - taskRunnerMap.remove(id); + TaskRunner taskRunner = taskRunnerMap.remove(id); + if (taskRunner != null) { + synchronized(taskRunnerCompleteCounter) { + ExecutionBlockId ebId = taskRunner.getContext().getExecutionBlockId(); + AtomicInteger counter = taskRunnerCompleteCounter.get(ebId); + if (counter != null) { + if (counter.decrementAndGet() <= 0) { + LOG.info(ebId + "'s all tasks are completed."); + workerContext.getHashShuffleAppenderManager().close(ebId); + taskRunnerCompleteCounter.remove(ebId); + } + } + } + } } if(workerContext.isYarnContainerMode()) { stop(); @@ -146,6 +161,7 @@ public int getNumTasks() { } } + Map taskRunnerCompleteCounter = new HashMap(); public void startTask(final String[] params) { //TODO change to use event dispatcher Thread t = new Thread() { @@ -162,6 +178,16 @@ public void run() { taskRunnerHistoryMap.put(taskRunner.getId(), taskRunner.getContext().getExcutionBlockHistory()); } + synchronized(taskRunnerCompleteCounter) { + ExecutionBlockId ebId = taskRunner.getContext().getExecutionBlockId(); + AtomicInteger counter = taskRunnerCompleteCounter.get(ebId); + if (counter == null) { + counter = new AtomicInteger(0); + taskRunnerCompleteCounter.put(ebId, counter); + } + counter.incrementAndGet(); + + } taskRunner.init(systemConf); taskRunner.start(); } catch (Exception e) { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java index a823d2bb9b..200712cfdf 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java @@ -25,15 +25,13 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.tajo.LocalTajoTestingUtility; -import org.apache.tajo.QueryUnitAttemptId; -import org.apache.tajo.TajoConstants; -import org.apache.tajo.TajoTestingCluster; +import org.apache.tajo.*; import org.apache.tajo.algebra.Expr; import org.apache.tajo.catalog.*; import org.apache.tajo.catalog.proto.CatalogProtos.StoreType; import org.apache.tajo.common.TajoDataTypes.Type; import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.conf.TajoConf.ConfVars; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.datum.NullDatum; @@ -64,6 +62,7 @@ import java.io.File; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; @@ -532,9 +531,9 @@ public final void testPartitionedStorePlan() throws IOException, PlanningExcepti FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), Integer.MAX_VALUE); QueryUnitAttemptId id = LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan); - Path workDir = CommonTestingUtil.getTestDir("target/test-data/testPartitionedStorePlan"); TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(), - id, new FileFragment[] { frags[0] }, workDir); + id, new FileFragment[] { frags[0] }, + CommonTestingUtil.getTestDir("target/test-data/testPartitionedStorePlan")); ctx.setEnforcer(new Enforcer()); Expr context = analyzer.parse(QUERIES[7]); LogicalPlan plan = planner.createPlan(session, context); @@ -551,27 +550,36 @@ public final void testPartitionedStorePlan() throws IOException, PlanningExcepti TableMeta outputMeta = CatalogUtil.newTableMeta(dataChannel.getStoreType()); FileSystem fs = sm.getFileSystem(); + QueryId queryId = id.getQueryUnitId().getExecutionBlockId().getQueryId(); + ExecutionBlockId ebId = id.getQueryUnitId().getExecutionBlockId(); PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm); PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode); exec.init(); exec.next(); exec.close(); + ctx.getHashShuffleAppenderManager().close(ebId); - Path path = new Path(workDir, "output"); - FileStatus [] list = fs.listStatus(path); - assertEquals(numPartitions, list.length); + String executionBlockBaseDir = queryId.toString() + "/output" + "/" + ebId.getId() + "/hash-shuffle"; + Path queryLocalTmpDir = new Path(conf.getVar(ConfVars.WORKER_TEMPORAL_DIR) + "/" + executionBlockBaseDir); + FileStatus [] list = fs.listStatus(queryLocalTmpDir); - FileFragment[] fragments = new FileFragment[list.length]; - int i = 0; + List fragments = new ArrayList(); for (FileStatus status : list) { - fragments[i++] = new FileFragment("partition", status.getPath(), 0, status.getLen()); + assertTrue(status.isDirectory()); + FileStatus [] files = fs.listStatus(status.getPath()); + for (FileStatus eachFile: files) { + fragments.add(new FileFragment("partition", eachFile.getPath(), 0, eachFile.getLen())); + } } +// CommonTestingUtil.getTestDir("target/test-data/testPartitionedStorePlan") + + assertEquals(numPartitions, fragments.size()); Scanner scanner = new MergeScanner(conf, rootNode.getOutSchema(), outputMeta, TUtil.newList(fragments)); scanner.init(); Tuple tuple; - i = 0; + int i = 0; while ((tuple = scanner.next()) != null) { assertEquals(6, tuple.get(2).asInt4()); // sum assertEquals(3, tuple.get(3).asInt4()); // max diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java index b59c49515b..fbd1ae35f8 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java @@ -90,9 +90,9 @@ public TestJoinQuery(String joinOption) { public static Collection generateParameters() { return Arrays.asList(new Object[][]{ {"Hash_NoBroadcast"}, - {"Sort_NoBroadcast"}, - {"Hash"}, - {"Sort"}, +// {"Sort_NoBroadcast"}, +// {"Hash"}, +// {"Sort"}, }); } diff --git a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java index e340953fa7..f98873517e 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java @@ -57,7 +57,7 @@ public final void case1() throws Exception { // tpch/lineitem.tbl long[] expectedNumRows = new long[]{5, 2, 2, 2}; long[] expectedNumBytes = new long[]{604, 18, 18, 8}; - long[] expectedReadBytes = new long[]{604, 0, 18, 0}; + long[] expectedReadBytes = new long[]{604, 18, 18, 0}; assertStatus(2, expectedNumRows, expectedNumBytes, expectedReadBytes); } finally { @@ -75,7 +75,7 @@ public final void case2() throws Exception { // tpch/lineitem.tbl long[] expectedNumRows = new long[]{5, 2, 2, 2, 2, 2}; long[] expectedNumBytes = new long[]{604, 162, 162, 138, 138, 194}; - long[] expectedReadBytes = new long[]{604, 0, 162, 0, 138, 0}; + long[] expectedReadBytes = new long[]{604, 162, 162, 0, 138, 0}; assertStatus(3, expectedNumRows, expectedNumBytes, expectedReadBytes); } finally { @@ -106,7 +106,7 @@ public final void case3() throws Exception { // in/out * subquery(4) long[] expectedNumRows = new long[]{2, 2, 5, 5, 7, 2, 2, 2}; long[] expectedNumBytes = new long[]{8, 34, 20, 75, 109, 34, 34, 18}; - long[] expectedReadBytes = new long[]{8, 0, 20, 0, 109, 0, 34, 0}; + long[] expectedReadBytes = new long[]{8, 34, 20, 75, 109, 0, 34, 0}; assertStatus(4, expectedNumRows, expectedNumBytes, expectedReadBytes); } finally { diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java b/tajo-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java index a2dda76688..c6be73bc02 100644 --- a/tajo-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java +++ b/tajo-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java @@ -172,7 +172,6 @@ public Appender getAppender(TableMeta meta, Schema schema, Path path) return appender; } - public TableMeta getTableMeta(Path tablePath) throws IOException { TableMeta meta; diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java new file mode 100644 index 0000000000..de6eb62ff7 --- /dev/null +++ b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java @@ -0,0 +1,99 @@ +/** + * 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.tajo.storage; + +import org.apache.tajo.catalog.statistics.TableStats; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +public class HashShuffleAppender implements Appender { + private FileAppender appender; + private AtomicBoolean closed = new AtomicBoolean(false); + private int partId; + + private TableStats tableStats; + + public HashShuffleAppender(int partId, FileAppender appender) { + this.partId = partId; + this.appender = appender; + } + + @Override + public void init() throws IOException { + } + + public long addTuples(List tuples) throws IOException { + synchronized(appender) { + if (closed.get()) { + return 0; + } + long currentPos = appender.getOffset(); + + for (Tuple eachTuple: tuples) { + appender.addTuple(eachTuple); + } + return appender.getOffset() - currentPos; + } + } + + @Override + public void addTuple(Tuple t) throws IOException { + synchronized(appender) { + if (closed.get()) { + return; + } + appender.addTuple(t); + } + } + + @Override + public void flush() throws IOException { + synchronized(appender) { + if (closed.get()) { + return; + } + appender.flush(); + } + } + + @Override + public void close() throws IOException { + synchronized(appender) { + if (closed.get()) { + return; + } + appender.close(); + closed.set(true); + tableStats = appender.getStats(); + } + } + + @Override + public void enableStats() { + } + + @Override + public TableStats getStats() { + synchronized(appender) { + return appender.getStats(); + } + } +} diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java new file mode 100644 index 0000000000..791d076abe --- /dev/null +++ b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java @@ -0,0 +1,169 @@ +/** + * 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.tajo.storage; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.Path; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.conf.TajoConf.ConfVars; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class HashShuffleAppenderManager { + private static final Log LOG = LogFactory.getLog(HashShuffleAppenderManager.class); + + private Map> appenderMap = + new ConcurrentHashMap>(); + private TajoConf systemConf; + private FileSystem defaultFS; + private FileSystem localFS; + private LocalDirAllocator lDirAllocator; + + public HashShuffleAppenderManager(TajoConf systemConf) throws IOException { + this.systemConf = systemConf; + + // initialize LocalDirAllocator + lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); + + // initialize DFS and LocalFileSystems + defaultFS = TajoConf.getTajoRootDir(systemConf).getFileSystem(systemConf); + localFS = FileSystem.getLocal(systemConf); + + } + + public HashShuffleAppender getAppender(TajoConf tajoConf, ExecutionBlockId ebId, int partId, + TableMeta meta, Schema outSchema) throws IOException { + synchronized (appenderMap) { + Map partitionAppenderMap = appenderMap.get(ebId); + + if (partitionAppenderMap == null) { + partitionAppenderMap = new ConcurrentHashMap(); + appenderMap.put(ebId, partitionAppenderMap); + } + + PartitionAppenderMeta partitionAppenderMeta = partitionAppenderMap.get(partId); + if (partitionAppenderMeta == null) { + Path dataFile = getDataFile(ebId, partId); + FileSystem fs = dataFile.getFileSystem(systemConf); + if (fs.exists(dataFile)) { + FileStatus status = fs.getFileStatus(dataFile); + LOG.info("File " + dataFile + " already exists, size=" + status.getLen()); + } + + if (!fs.exists(dataFile.getParent())) { + fs.mkdirs(dataFile.getParent()); + } + FileAppender appender = (FileAppender)StorageManagerFactory.getStorageManager( + tajoConf).getAppender(meta, outSchema, dataFile); + appender.enableStats(); + appender.init(); + + partitionAppenderMeta = new PartitionAppenderMeta(); + partitionAppenderMeta.partId = partId; + partitionAppenderMeta.dataFile = dataFile; + partitionAppenderMeta.appender = new HashShuffleAppender(partId, appender); + partitionAppenderMap.put(partId, partitionAppenderMeta); + + LOG.info("Create Hash shuffle file(partId=" + partId + "): " + dataFile); + } + + return partitionAppenderMeta.appender; + } + } + + public static int getPartParentId(int partId, TajoConf tajoConf) { + return partId % tajoConf.getIntVar(TajoConf.ConfVars.HASH_SHUFFLE_PARENT_DIRS); + } + + private Path getDataFile(ExecutionBlockId ebId, int partId) throws IOException { + try { + // the base dir for an output dir + String executionBlockBaseDir = ebId.getQueryId().toString() + "/output" + "/" + ebId.getId() + "/hash-shuffle"; + Path baseDirPath = localFS.makeQualified(lDirAllocator.getLocalPathForWrite(executionBlockBaseDir, systemConf)); + LOG.info(ebId + "'s basedir is created (" + baseDirPath + ")"); + + // If EB has many partition, too many shuffle file are in single directory. + return StorageUtil.concatPath(baseDirPath, "" + getPartParentId(partId, systemConf), "" + partId); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + throw new IOException(e); + } + } + + public Path getPartitionAppenderDataFile(ExecutionBlockId ebId, int partId) { + synchronized (appenderMap) { + Map partitionAppenderMap = appenderMap.get(ebId); + if (partitionAppenderMap != null) { + PartitionAppenderMeta meta = partitionAppenderMap.get(partId); + if (meta != null) { + return meta.dataFile; + } + } + } + + LOG.warn("Can't find HashShuffleAppender:" + ebId + ", part=" + partId); + return null; + } + + public void close(ExecutionBlockId ebId) { + Map partitionAppenderMap = null; + synchronized (appenderMap) { + partitionAppenderMap = appenderMap.remove(ebId); + } + + if (partitionAppenderMap == null) { + return; + } + + LOG.info("Close HashShuffleAppender:" + ebId); + for (PartitionAppenderMeta eachMeta : partitionAppenderMap.values()) { + try { + eachMeta.appender.close(); + } catch (IOException e) { + } + } + } + + static class PartitionAppenderMeta { + int partId; + HashShuffleAppender appender; + Path dataFile; + + public int getPartId() { + return partId; + } + + public HashShuffleAppender getAppender() { + return appender; + } + + public Path getDataFile() { + return dataFile; + } + } +} diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java index 41d1e05451..1f57675bc2 100644 --- a/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java +++ b/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java @@ -729,6 +729,7 @@ public void close() throws IOException { @Override public TableStats getStats() { if (enabledStats) { + stats.setNumBytes(pos); return stats.getTableStat(); } else { return null; diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java index 12cd1a3633..29a1e16aa2 100644 --- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java +++ b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java @@ -47,6 +47,7 @@ import org.apache.tajo.pullserver.listener.FileCloseListener; import org.apache.tajo.pullserver.retriever.FileChunk; import org.apache.tajo.rpc.RpcChannelFactory; +import org.apache.tajo.storage.HashShuffleAppenderManager; import org.apache.tajo.storage.RowStoreUtil; import org.apache.tajo.storage.RowStoreUtil.RowStoreDecoder; import org.apache.tajo.storage.Tuple; @@ -208,10 +209,13 @@ public void init(Configuration conf) { selector = RpcChannelFactory.createServerChannelFactory("PullServerAuxService", workerNum); localFS = new LocalFileSystem(); - super.init(new Configuration(conf)); + //super.init(new Configuration(conf)); + super.init(conf); this.getConfig().setInt(TajoConf.ConfVars.PULLSERVER_PORT.varname , TajoConf.ConfVars.PULLSERVER_PORT.defaultIntVal); + + LOG.info("Tajo PullServer initialized: readaheadLength=" + readaheadLength); } catch (Throwable t) { LOG.error(t); } @@ -229,9 +233,11 @@ public synchronized void start() { throw new RuntimeException(ex); } bootstrap.setPipelineFactory(pipelineFact); + port = conf.getInt(ConfVars.PULLSERVER_PORT.varname, ConfVars.PULLSERVER_PORT.defaultIntVal); Channel ch = bootstrap.bind(new InetSocketAddress(port)); + accepted.add(ch); port = ((InetSocketAddress)ch.getLocalAddress()).getPort(); conf.set(ConfVars.PULLSERVER_PORT.varname, Integer.toString(port)); @@ -375,9 +381,8 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) final List subQueryIds = params.get("sid"); final List partIds = params.get("p"); - if (types == null || taskIdList == null || subQueryIds == null || qids == null - || partIds == null) { - sendError(ctx, "Required queryId, type, taskIds, subquery Id, and part id", + if (types == null || subQueryIds == null || qids == null || partIds == null) { + sendError(ctx, "Required queryId, type, subquery Id, and part id", BAD_REQUEST); return; } @@ -388,12 +393,15 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) return; } - final List chunks = Lists.newArrayList(); - + String partId = partIds.get(0); String queryId = qids.get(0); String shuffleType = types.get(0); String sid = subQueryIds.get(0); - String partId = partIds.get(0); + + if (!shuffleType.equals("h") && !shuffleType.equals("s") && taskIdList == null) { + sendError(ctx, "Required taskIds", BAD_REQUEST); + } + List taskIds = splitMaps(taskIdList); LOG.info("PullServer request param: shuffleType=" + shuffleType + @@ -404,6 +412,8 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) LOG.info("PullServer baseDir: " + conf.get(ConfVars.WORKER_TEMPORAL_DIR.varname) + "/" + queryBaseDir); + final List chunks = Lists.newArrayList(); + // if a subquery requires a range shuffle if (shuffleType.equals("r")) { String ta = taskIds.get(0); @@ -432,18 +442,19 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) // if a subquery requires a hash shuffle or a scattered hash shuffle } else if (shuffleType.equals("h") || shuffleType.equals("s")) { - for (String ta : taskIds) { - if (!lDirAlloc.ifExists(queryBaseDir + "/" + sid + "/" + ta + "/output/" + partId, conf)) { - LOG.warn(e); - sendError(ctx, NO_CONTENT); - return; - } - Path path = localFS.makeQualified( - lDirAlloc.getLocalPathToRead(queryBaseDir + "/" + sid + "/" + ta + "/output/" + partId, conf)); - File file = new File(path.toUri()); - FileChunk chunk = new FileChunk(file, 0, file.length()); - chunks.add(chunk); + int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), (TajoConf) conf); + String partPath = queryBaseDir + "/" + sid + "/hash-shuffle/" + partParentId + "/" + partId; + if (!lDirAlloc.ifExists(partPath, conf)) { + LOG.warn("Partition shuffle file not exists: " + partPath); + sendError(ctx, NO_CONTENT); + return; } + + Path path = localFS.makeQualified(lDirAlloc.getLocalPathToRead(partPath, conf)); + + File file = new File(path.toUri()); + FileChunk chunk = new FileChunk(file, 0, file.length()); + chunks.add(chunk); } else { LOG.error("Unknown shuffle type: " + shuffleType); sendError(ctx, "Unknown shuffle type:" + shuffleType, BAD_REQUEST); From 06045064ec32b6111ece0abf7343402e419ca608 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=EA=B9=80=ED=98=95=EC=A4=80?= Date: Wed, 6 Aug 2014 22:57:35 +0900 Subject: [PATCH 2/8] TAJO-992: Reduce number of hash shuffle output file. --- .../master/querymaster/Repartitioner.java | 5 ++-- .../java/org/apache/tajo/worker/Fetcher.java | 26 ++++++++++--------- 2 files changed, 17 insertions(+), 14 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java index 4d8d8f5161..f407186fc6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java @@ -761,7 +761,6 @@ public static void scheduleHashShuffledFetches(TaskSchedulerContext schedulerCon fragments.add(frag); SubQuery.scheduleFragments(subQuery, fragments); - Map> hashedByHost; Map finalFetches = new HashMap(); Map> intermediates = new HashMap>(); @@ -782,6 +781,7 @@ public static void scheduleHashShuffledFetches(TaskSchedulerContext schedulerCon } } } + // merge intermediate data by partId, pullHost List partitions = new ArrayList(); for (List eachList: intermediatesByPartAndHost.values()) { @@ -808,9 +808,10 @@ public static void scheduleHashShuffledFetches(TaskSchedulerContext schedulerCon } } + // make FetchImpl per PullServer, PartId Map> hashed = hashByKey(partitions); for (Entry> interm : hashed.entrySet()) { - hashedByHost = hashByHost(interm.getValue()); + Map> hashedByHost = hashByHost(interm.getValue()); for (Entry> e : hashedByHost.entrySet()) { FetchImpl fetch = new FetchImpl(e.getKey(), channel.getShuffleType(), diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java b/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java index 4cb7dbe697..aa22bb858f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java @@ -232,8 +232,16 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) HttpChunk chunk = (HttpChunk) e.getMessage(); if (chunk.isLast()) { readingChunks = false; + long fileLength = file.length(); + if (fileLength == length) { + LOG.info("Data fetch is done (total received bytes: " + fileLength + + ")"); + } else { + LOG.info("Data fetch is done, but cannot get all data " + + "(received/total: " + fileLength + "/" + length + ")"); + } } else { - if(fc != null) { + if(fc != null){ fc.write(chunk.getContent().toByteBuffer()); } } @@ -242,17 +250,11 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) if(raf != null) { fileLen = file.length(); } - if (!readingChunks) { - if (fileLen == length) { - LOG.info("Data fetch is done (total received bytes: " + fileLen - + ", number of received message: " + messageReceiveCount + "), file=" + file); - IOUtils.cleanup(LOG, fc, raf); - finishTime = System.currentTimeMillis(); - state = TajoProtos.FetcherState.FETCH_FINISHED; - } else { - LOG.info("Data fetch is done, but cannot get all data " - + "(received/total: " + fileLen + "/" + length + ")"); - } + + if(fileLen == length){ + IOUtils.cleanup(LOG, fc, raf); + finishTime = System.currentTimeMillis(); + state = TajoProtos.FetcherState.FETCH_FINISHED; } } } From 028f498eb18c9094b8ac7641d628ec58e3ffb605 Mon Sep 17 00:00:00 2001 From: HyoungJun Kim Date: Tue, 12 Aug 2014 06:37:52 +0900 Subject: [PATCH 3/8] TAJO-992: Reduce number of hash shuffle output file. Splittable IntermediateEntry. --- .../java/org/apache/tajo/conf/TajoConf.java | 2 + .../main/java/org/apache/tajo/util/Pair.java | 32 +- .../physical/HashShuffleFileWriteExec.java | 55 +-- .../tajo/master/querymaster/QueryMaster.java | 50 ++- .../tajo/master/querymaster/QueryUnit.java | 79 +++++ .../master/querymaster/Repartitioner.java | 182 +++++----- .../tajo/master/querymaster/SubQuery.java | 44 ++- .../org/apache/tajo/worker/FetchImpl.java | 32 ++ .../org/apache/tajo/worker/TajoWorker.java | 68 +++- .../tajo/worker/TajoWorkerManagerService.java | 15 +- .../java/org/apache/tajo/worker/Task.java | 2 +- .../apache/tajo/worker/TaskRunnerManager.java | 49 +-- .../src/main/proto/QueryMasterProtocol.proto | 1 + .../src/main/proto/TajoWorkerProtocol.proto | 31 +- .../tajo/engine/query/TestGroupByQuery.java | 8 +- .../tajo/engine/query/TestJoinQuery.java | 6 +- .../engine/query/TestTablePartitions.java | 73 +++- .../apache/tajo/master/TestRepartitioner.java | 315 ++++++++++++++++-- .../querymaster/TestIntermediateEntry.java | 53 +++ .../TestQueryUnitStatusUpdate.java | 6 +- .../tajo/storage/HashShuffleAppender.java | 121 ++++++- .../storage/HashShuffleAppenderManager.java | 83 ++++- .../pullserver/TajoPullServerService.java | 17 +- 23 files changed, 1082 insertions(+), 242 deletions(-) create mode 100644 tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestIntermediateEntry.java diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index 942a73516e..dac3a93b0d 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -220,6 +220,8 @@ public static enum ConfVars { SHUFFLE_FETCHER_CHUNK_MAX_SIZE("tajo.shuffle.fetcher.chunk.max-size", 8192), SHUFFLE_FETCHER_READ_TIMEOUT("tajo.shuffle.fetcher.read.timeout-sec", 120), SHUFFLE_FETCHER_READ_RETRY_MAX_NUM("tajo.shuffle.fetcher.read.retry.max-num", 20), + SHUFFLE_HASH_APPENDER_BUFFER_SIZE("tajo.shuffle.hash.appender.buffer.size", 10000), + SHUFFLE_HASH_APPENDER_PAGE_VOLUME("tajo.shuffle.hash.appender.page.volumn-mb", 30), ////////////////////////////////// // Storage Configuration diff --git a/tajo-common/src/main/java/org/apache/tajo/util/Pair.java b/tajo-common/src/main/java/org/apache/tajo/util/Pair.java index 72cfc5c60a..7b5f8adceb 100644 --- a/tajo-common/src/main/java/org/apache/tajo/util/Pair.java +++ b/tajo-common/src/main/java/org/apache/tajo/util/Pair.java @@ -19,19 +19,37 @@ package org.apache.tajo.util; public class Pair { - private final T1 value1; - private final T2 value2; + private T1 first; + private T2 second; - public Pair(T1 value1, T2 value2) { - this.value1 = value1; - this.value2 = value2; + public Pair(T1 first, T2 second) { + this.first = first; + this.second = second; } public T1 getFirst() { - return value1; + return first; } public T2 getSecond() { - return value2; + return second; + } + + public void setFirst(T1 first) { + this.first = first; + } + + public void setSecond(T2 second) { + this.second = second; + } + + @Override + public String toString() { + return first + "," + second; + } + + @Override + public int hashCode() { + return toString().hashCode(); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java index 2ac36fe9df..281de6d5e3 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java @@ -24,8 +24,8 @@ import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.Column; import org.apache.tajo.catalog.TableMeta; -import org.apache.tajo.catalog.statistics.StatisticsUtil; import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.conf.TajoConf.ConfVars; import org.apache.tajo.engine.planner.logical.ShuffleFileWriteNode; import org.apache.tajo.storage.AbstractStorageManager; import org.apache.tajo.storage.HashShuffleAppender; @@ -53,6 +53,7 @@ public final class HashShuffleFileWriteExec extends UnaryPhysicalExec { private final int numShuffleOutputs; private final int [] shuffleKeyIds; private HashShuffleAppenderManager hashShuffleAppenderManager; + private int numHashShuffleBufferTuples; public HashShuffleFileWriteExec(TaskAttemptContext context, final AbstractStorageManager sm, final ShuffleFileWriteNode plan, final PhysicalExec child) throws IOException { @@ -74,6 +75,7 @@ public HashShuffleFileWriteExec(TaskAttemptContext context, final AbstractStorag } this.partitioner = new HashPartitioner(shuffleKeyIds, numShuffleOutputs); this.hashShuffleAppenderManager = context.getHashShuffleAppenderManager(); + this.numHashShuffleBufferTuples = context.getConf().getIntVar(ConfVars.SHUFFLE_HASH_APPENDER_BUFFER_SIZE); } @Override @@ -91,8 +93,9 @@ private HashShuffleAppender getAppender(int partId) throws IOException { return appender; } - Map partitionStats = new HashMap(); +// Map partitionStats = new HashMap(); Map> partitionTuples = new HashMap>(); + long writtenBytes = 0L; @Override public Tuple next() throws IOException { @@ -115,17 +118,12 @@ public Tuple next() throws IOException { partitionTupleList.add(tuple.clone()); } catch (CloneNotSupportedException e) { } - if (tupleCount >= 10000) { + if (tupleCount >= numHashShuffleBufferTuples) { for (Map.Entry> entry : partitionTuples.entrySet()) { int appendPartId = entry.getKey(); HashShuffleAppender appender = getAppender(appendPartId); - long appendedSize = appender.addTuples(entry.getValue()); - Long previousSize = partitionStats.get(appendPartId); - if (previousSize == null) { - partitionStats.put(appendPartId, appendedSize); - } else { - partitionStats.put(appendPartId, appendedSize + previousSize); - } + int appendedSize = appender.addTuples(context.getTaskId(), entry.getValue()); + writtenBytes += appendedSize; entry.getValue().clear(); } tupleCount = 0; @@ -136,45 +134,16 @@ public Tuple next() throws IOException { for (Map.Entry> entry : partitionTuples.entrySet()) { int appendPartId = entry.getKey(); HashShuffleAppender appender = getAppender(appendPartId); - long appendedSize = appender.addTuples(entry.getValue()); - Long previousSize = partitionStats.get(appendPartId); - if (previousSize == null) { - partitionStats.put(appendPartId, appendedSize); - } else { - partitionStats.put(appendPartId, appendedSize + previousSize); - } + int appendedSize = appender.addTuples(context.getTaskId(), entry.getValue()); + writtenBytes += appendedSize; entry.getValue().clear(); } - // set table stats - List statSet = new ArrayList(); - for (Integer eachPartId : partitionStats.keySet()) { - HashShuffleAppender appender = appenderMap.get(eachPartId); - TableStats appenderStat = appender.getStats(); - TableStats tableStats = null; - try { - tableStats = (TableStats) appenderStat.clone(); - } catch (CloneNotSupportedException e) { - LOG.error(e); - } - tableStats.setNumBytes(partitionStats.get(eachPartId)); - tableStats.setReadBytes(partitionStats.get(eachPartId)); - - if (numRows > 0) { - context.addShuffleFileOutput(eachPartId, - hashShuffleAppenderManager.getPartitionAppenderDataFile( - context.getQueryId().getQueryUnitId().getExecutionBlockId(), eachPartId).getName()); - context.addPartitionOutputVolume(eachPartId, tableStats.getNumBytes()); - } - statSet.add(tableStats); - } - - // Collect and aggregated statistics data - TableStats aggregated = StatisticsUtil.aggregateTableStat(statSet); + TableStats aggregated = (TableStats)child.getInputStats().clone(); + aggregated.setReadBytes(writtenBytes); aggregated.setNumRows(numRows); context.setResultStats(aggregated); - partitionStats.clear(); partitionTuples.clear(); return null; diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java index 25af82f56a..3a6431abf4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java @@ -35,6 +35,8 @@ import org.apache.tajo.engine.planner.global.GlobalPlanner; import org.apache.tajo.ipc.TajoMasterProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol.ExecutionBlockReport; +import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto; import org.apache.tajo.master.TajoAsyncDispatcher; import org.apache.tajo.master.event.QueryStartEvent; import org.apache.tajo.rpc.CallFuture; @@ -52,6 +54,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.tajo.ipc.TajoMasterProtocol.TajoHeartbeat; @@ -164,13 +167,19 @@ public void stop() { } } - protected void cleanupExecutionBlock(List executionBlockIds) { - LOG.info("cleanup executionBlocks : " + executionBlockIds); + + protected List cleanupExecutionBlock(SubQuery subQuery, + List executionBlockIds) throws Exception { + LOG.info("cleanup executionBlocks: " + executionBlockIds); NettyClientBase rpc = null; List workers = getAllWorker(); TajoWorkerProtocol.ExecutionBlockListProto.Builder builder = TajoWorkerProtocol.ExecutionBlockListProto.newBuilder(); builder.addAllExecutionBlockId(Lists.newArrayList(executionBlockIds)); TajoWorkerProtocol.ExecutionBlockListProto executionBlockListProto = builder.build(); + + Exception err = null; + + List intermediateEntries = new ArrayList(); for (TajoMasterProtocol.WorkerResourceProto worker : workers) { try { if (worker.getPeerRpcPort() == 0) continue; @@ -179,13 +188,48 @@ protected void cleanupExecutionBlock(List ex TajoWorkerProtocol.class, true); TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub(); - tajoWorkerProtocolService.cleanupExecutionBlocks(null, executionBlockListProto, NullCallback.get()); + CallFuture callBack = new CallFuture(); + tajoWorkerProtocolService.cleanupExecutionBlocks(null, executionBlockListProto, callBack); + long startTime = System.currentTimeMillis(); + if (err == null) { + while (true) { + try { + ExecutionBlockReport report = callBack.get(10, TimeUnit.SECONDS); + if (report == null) { + if (System.currentTimeMillis() - startTime > 180 * 1000) { + LOG.error("Can't ExecutionBlock report from " + worker.getHost() + ":" + worker.getPeerRpcPort()); + break; + } + } + if (!report.getReportSuccess()) { + throw new Exception("Getting ExecutionBlockReport from " + worker.getHost() + ":" + worker.getPeerRpcPort() + + " error: " + report.getReportErrorMessage()); + } + intermediateEntries.addAll(report.getIntermediateEntriesList()); + break; + } catch (InterruptedException e) { + LOG.error(e.getMessage(), e); + break; + } catch (TimeoutException e) { + if (System.currentTimeMillis() - startTime > 180 * 1000) { + LOG.error("Can't ExecutionBlock report from " + worker.getHost() + ":" + worker.getPeerRpcPort()); + break; + } + } + } + } } catch (Exception e) { LOG.error(e.getMessage()); + err = e; } finally { connPool.releaseConnection(rpc); } } + if (err != null) { + throw err; + } + + return intermediateEntries; } private void cleanup(QueryId queryId) { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java index 9d308350f4..f41fd0ee15 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java @@ -34,12 +34,15 @@ import org.apache.tajo.QueryUnitId; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.engine.planner.logical.*; +import org.apache.tajo.ipc.TajoWorkerProtocol.FailureIntermediateProto; +import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto; import org.apache.tajo.master.FragmentPair; import org.apache.tajo.master.TaskState; import org.apache.tajo.master.event.*; import org.apache.tajo.master.event.QueryUnitAttemptScheduleEvent.QueryUnitAttemptScheduleContext; import org.apache.tajo.storage.DataLocation; import org.apache.tajo.storage.fragment.FileFragment; +import org.apache.tajo.util.Pair; import org.apache.tajo.util.TajoIdUtils; import org.apache.tajo.worker.FetchImpl; @@ -667,6 +670,11 @@ public PullHost clone() throws CloneNotSupportedException { newPullHost.port = port; return newPullHost; } + + @Override + public String toString() { + return host + ":" + port; + } } public static class IntermediateEntry { @@ -676,6 +684,31 @@ public static class IntermediateEntry { int partId; PullHost host; long volume; + List> pages; + List>> failureRowNums; + + public IntermediateEntry(IntermediateEntryProto proto) { + this.ebId = new ExecutionBlockId(proto.getEbId()); + this.taskId = proto.getTaskId(); + this.attemptId = proto.getAttemptId(); + this.partId = proto.getPartId(); + + String[] pullHost = proto.getHost().split(":"); + this.host = new PullHost(pullHost[0], Integer.parseInt(pullHost[1])); + this.volume = proto.getVolume(); + + failureRowNums = new ArrayList>>(); + for (FailureIntermediateProto eachFailure: proto.getFailuresList()) { + + failureRowNums.add(new Pair(eachFailure.getPagePos(), + new Pair(eachFailure.getStartRowNum(), eachFailure.getEndRowNum()))); + } + + pages = new ArrayList>(); + for (IntermediateEntryProto.PageProto eachPage: proto.getPagesList()) { + pages.add(new Pair(eachPage.getPos(), eachPage.getLength())); + } + } public IntermediateEntry(int taskId, int attemptId, int partId, PullHost host) { this.taskId = taskId; @@ -724,9 +757,55 @@ public long setVolume(long volume) { return this.volume = volume; } + public List> getPages() { + return pages; + } + + public void setPages(List> pages) { + this.pages = pages; + } + + public List>> getFailureRowNums() { + return failureRowNums; + } + @Override public int hashCode() { return Objects.hashCode(ebId, taskId, partId, attemptId, host); } + + public List> split(long firstSplitVolume, long splitVolume) { + List> splits = new ArrayList>(); + + if (pages == null || pages.isEmpty()) { + return splits; + } + int pageSize = pages.size(); + + long currentOffset = -1; + long currentBytes = 0; + + long realSplitVolume = firstSplitVolume > 0 ? firstSplitVolume : splitVolume; + for (int i = 0; i < pageSize; i++) { + Pair eachPage = pages.get(i); + if (currentOffset == -1) { + currentOffset = eachPage.getFirst(); + } + if (currentBytes > 0 && currentBytes + eachPage.getSecond() >= realSplitVolume) { + splits.add(new Pair(currentOffset, currentBytes)); + currentOffset = eachPage.getFirst(); + currentBytes = 0; + realSplitVolume = splitVolume; + } + + currentBytes += eachPage.getSecond(); + } + + //add last + if (currentBytes > 0) { + splits.add(new Pair(currentOffset, currentBytes)); + } + return splits; + } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java index f407186fc6..df462faebe 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java @@ -212,8 +212,6 @@ public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerC LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join with all tables, base_table=%s, base_volume=%d", scans[baseScanIdx].getCanonicalName(), stats[baseScanIdx])); scheduleLeafTasksWithBroadcastTable(schedulerContext, subQuery, baseScanIdx, fragments); - - } else if (!execBlock.getBroadcastTables().isEmpty()) { // If some relations of this EB are broadcasted boolean hasNonLeafNode = false; List largeScanIndexList = new ArrayList(); @@ -250,7 +248,7 @@ public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerC for (Integer eachId : largeScanIndexList) { largeTableNames += scans[eachId].getTableName() + ","; } - throw new IOException("Broadcase join with leaf node should have only one large table, " + + throw new IOException("Broadcast join with leaf node should have only one large table, " + "but " + largeScanIndexList.size() + ", tables=" + largeTableNames); } int baseScanIdx = largeScanIndexList.isEmpty() ? maxStatsScanIdx : largeScanIndexList.get(0); @@ -323,9 +321,12 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster scanEbId = childBlock.getId(); } SubQuery childExecSM = subQuery.getContext().getSubQuery(childBlock.getId()); - for (QueryUnit task : childExecSM.getQueryUnits()) { - if (task.getIntermediateData() != null && !task.getIntermediateData().isEmpty()) { - for (IntermediateEntry intermEntry : task.getIntermediateData()) { + //for (QueryUnit task : childExecSM.getQueryUnits()) { + // if (task.getIntermediateData() != null && !task.getIntermediateData().isEmpty()) { + // for (IntermediateEntry intermEntry : task.getIntermediateData()) { + if (childExecSM.getHashShuffleIntermediateEntries() != null && + !childExecSM.getHashShuffleIntermediateEntries().isEmpty()) { + for (IntermediateEntry intermEntry: childExecSM.getHashShuffleIntermediateEntries()) { intermEntry.setEbId(childBlock.getId()); if (hashEntries.containsKey(intermEntry.getPartId())) { Map> tbNameToInterm = @@ -360,17 +361,17 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster } } } - } +// } // merge intermediate entry by ebid, pullhost - Map>> mergedHashEntries = mergeIntermediateByPullHost(hashEntries); + // Map>> mergedHashEntries = mergeIntermediateByPullHost(hashEntries); // hashEntries can be zero if there are no input data. // In the case, it will cause the zero divided exception. // it avoids this problem. int[] avgSize = new int[2]; - avgSize[0] = mergedHashEntries.size() == 0 ? 0 : (int) (stats[0] / mergedHashEntries.size()); - avgSize[1] = mergedHashEntries.size() == 0 ? 0 : (int) (stats[1] / mergedHashEntries.size()); + avgSize[0] = hashEntries.size() == 0 ? 0 : (int) (stats[0] / hashEntries.size()); + avgSize[1] = hashEntries.size() == 0 ? 0 : (int) (stats[1] / hashEntries.size()); int bothFetchSize = avgSize[0] + avgSize[1]; // Getting the desire number of join tasks according to the volumn @@ -385,10 +386,10 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster // determine the number of task per 64MB int maxTaskNum = (int) Math.ceil((double) mb / desireJoinTaskVolumn); LOG.info("The calculated number of tasks is " + maxTaskNum); - LOG.info("The number of total shuffle keys is " + mergedHashEntries.size()); + LOG.info("The number of total shuffle keys is " + hashEntries.size()); // the number of join tasks cannot be larger than the number of // distinct partition ids. - int joinTaskNum = Math.min(maxTaskNum, mergedHashEntries.size()); + int joinTaskNum = Math.min(maxTaskNum, hashEntries.size()); LOG.info("The determined number of join tasks is " + joinTaskNum); FileFragment[] rightFragments = new FileFragment[1 + (broadcastFragments == null ? 0 : broadcastFragments.length)]; @@ -400,7 +401,7 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster // Assign partitions to tasks in a round robin manner. for (Entry>> entry - : mergedHashEntries.entrySet()) { + : hashEntries.entrySet()) { addJoinShuffle(subQuery, entry.getKey(), entry.getValue()); } @@ -413,7 +414,7 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster * @param hashEntries * @return */ - private static Map>> mergeIntermediateByPullHost( + public static Map>> mergeIntermediateByPullHost( Map>> hashEntries) { Map>> mergedHashEntries = new HashMap>>(); @@ -766,38 +767,8 @@ public static void scheduleHashShuffledFetches(TaskSchedulerContext schedulerCon List>(); for (ExecutionBlock block : masterPlan.getChilds(execBlock)) { - Map> intermediatesByPartAndHost = new HashMap>(); - for (QueryUnit tasks : subQuery.getContext().getSubQuery(block.getId()).getQueryUnits()) { - if (tasks.getIntermediateData() != null) { - for (IntermediateEntry eachEntry: tasks.getIntermediateData()) { - int partId = eachEntry.getPartId(); - String pullHost = eachEntry.getPullHost().getPullAddress(); - List intermediateList = intermediatesByPartAndHost.get(partId + pullHost); - if (intermediateList == null) { - intermediateList = new ArrayList(); - intermediatesByPartAndHost.put(partId + pullHost, intermediateList); - } - intermediateList.add(eachEntry); - } - } - } - - // merge intermediate data by partId, pullHost List partitions = new ArrayList(); - for (List eachList: intermediatesByPartAndHost.values()) { - if (eachList.isEmpty()) { - continue; - } - IntermediateEntry first = eachList.get(0); - IntermediateEntry mergedEntry = new IntermediateEntry(-1, -1, first.getPartId(), first.getPullHost()); - mergedEntry.setEbId(first.getEbId()); - long volume = 0; - for (IntermediateEntry eachEntry: eachList) { - volume += eachEntry.getVolume(); - } - mergedEntry.setVolume(volume); - partitions.add(mergedEntry); - } + partitions.addAll(subQuery.getContext().getSubQuery(block.getId()).getHashShuffleIntermediateEntries()); // In scattered hash shuffle, Collecting each IntermediateEntry if (channel.getShuffleType() == SCATTERED_HASH_SHUFFLE) { @@ -942,56 +913,47 @@ public static void scheduleFetchesByEvenDistributedVolumes(SubQuery subQuery, Ma public static void scheduleScatteredHashShuffleFetches(TaskSchedulerContext schedulerContext, SubQuery subQuery, Map> intermediates, String tableName) { - int i = 0; long splitVolume = ((long) 1048576) * subQuery.getContext().getConf(). getIntVar(ConfVars.DIST_QUERY_TABLE_PARTITION_VOLUME); // in bytes + long pageSize = ((long) 1048576) * subQuery.getContext().getConf(). + getIntVar(ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME); // in bytes + if (pageSize >= splitVolume) { + throw new RuntimeException("tajo.dist-query.table-partition.task-volume-mb should be great than " + + "tajo.shuffle.hash.appender.page.volumn-mb"); + } + List> fetches = new ArrayList>(); - long sumNumBytes = 0L; - Map> fetches = new HashMap>(); - - // Step 1 : divide fetch uris into the the proper number of tasks by - // SCATTERED_HASH_SHUFFLE_SPLIT_VOLUME for (Entry> listEntry : intermediates.entrySet()) { - - // Step 2: Sort IntermediateEntry by partition id. After first sort, - // we need to sort again by PullHost address because of data locality. - Collections.sort(listEntry.getValue(), new IntermediateEntryComparator()); - for (IntermediateEntry interm : listEntry.getValue()) { - FetchImpl fetch = new FetchImpl(interm.getPullHost(), SCATTERED_HASH_SHUFFLE, - listEntry.getKey(), interm.getPartId(), TUtil.newList(interm)); - if (fetches.size() == 0) { - fetches.put(i, TUtil.newList(fetch)); + // merge by PartitionId + Map> partitionIntermMap = new HashMap>(); + for (IntermediateEntry eachInterm: listEntry.getValue()) { + int partId = eachInterm.getPartId(); + List partitionInterms = partitionIntermMap.get(partId); + if (partitionInterms == null) { + partitionInterms = TUtil.newList(eachInterm); + partitionIntermMap.put(partId, partitionInterms); } else { + partitionInterms.add(eachInterm); + } + } - // Step 3: Compare current partition id with previous partition id because One task just - // can include one partitionId. - if (fetches.get(i).get(0).getPartitionId() != interm.getPartId()) { - i++; - fetches.put(i, TUtil.newList(fetch)); - sumNumBytes = 0L; - } else { - if ((sumNumBytes + interm.getVolume()) < splitVolume) { - fetches.get(i).add(fetch); - } else { - i++; - fetches.put(i, TUtil.newList(fetch)); - sumNumBytes = 0L; - } - } + // Grouping or splitting to fit DIST_QUERY_TABLE_PARTITION_VOLUME size + for (List partitionEntires : partitionIntermMap.values()) { + List> eachFetches = splitOrMergeIntermediates(listEntry.getKey(), partitionEntires, + splitVolume, pageSize); + if (eachFetches != null && !eachFetches.isEmpty()) { + fetches.addAll(eachFetches); } - sumNumBytes += interm.getVolume(); } } - // Step 4 : Set the proper number of tasks to the estimated task num schedulerContext.setEstimatedTaskNum(fetches.size()); - // Step 5 : Apply divided fetches - i = 0; + int i = 0; Map>[] fetchesArray = new Map[fetches.size()]; - for(Entry> entry : fetches.entrySet()) { + for(List entry : fetches) { fetchesArray[i] = new HashMap>(); - fetchesArray[i].put(tableName, entry.getValue()); + fetchesArray[i].put(tableName, entry); SubQuery.scheduleFetches(subQuery, fetchesArray[i]); i++; @@ -1002,17 +964,57 @@ public static void scheduleScatteredHashShuffleFetches(TaskSchedulerContext sche + ", DeterminedTaskNum : " + fetches.size()); } - static class IntermediateEntryComparator implements Comparator { + /** + * If a IntermediateEntry is large than splitVolume, List has single element. + * @param ebId + * @param entries + * @param splitVolume + * @return + */ + public static List> splitOrMergeIntermediates( + ExecutionBlockId ebId, List entries, long splitVolume, long pageSize) { + // Each List has splitVolume size. + List> fetches = new ArrayList>(); + + Iterator iter = entries.iterator(); + if (!iter.hasNext()) { + return null; + } + List fetchListForSingleTask = new ArrayList(); + long fetchListVolume = 0; + + while (iter.hasNext()) { + IntermediateEntry currentInterm = iter.next(); - @Override - public int compare(IntermediateEntry o1, IntermediateEntry o2) { - int cmp = Ints.compare(o1.getPartId(), o2.getPartId()); - if (cmp != 0) { - return cmp; + long firstSplitVolume = splitVolume - fetchListVolume; + if (firstSplitVolume < pageSize) { + firstSplitVolume = splitVolume; + } + List> splits = currentInterm.split(firstSplitVolume, splitVolume); + if (splits == null || splits.isEmpty()) { + break; } - return o1.getPullHost().getHost().compareTo(o2.getPullHost().getHost()); + for (Pair eachSplit: splits) { + if (fetchListVolume > 0 && fetchListVolume + eachSplit.getSecond() >= splitVolume) { + if (!fetchListForSingleTask.isEmpty()) { + fetches.add(fetchListForSingleTask); + } + fetchListForSingleTask = new ArrayList(); + fetchListVolume = 0; + } + FetchImpl fetch = new FetchImpl(currentInterm.getPullHost(), SCATTERED_HASH_SHUFFLE, + ebId, currentInterm.getPartId(), TUtil.newList(currentInterm)); + fetch.setOffset(eachSplit.getFirst()); + fetch.setLength(eachSplit.getSecond()); + fetchListForSingleTask.add(fetch); + fetchListVolume += eachSplit.getSecond(); + } + } + if (!fetchListForSingleTask.isEmpty()) { + fetches.add(fetchListForSingleTask); } + return fetches; } public static List createFetchURL(FetchImpl fetch, boolean includeParts) { @@ -1032,6 +1034,10 @@ public static List createFetchURL(FetchImpl fetch, boolean includeParts) { urlPrefix.append("s"); } + if (fetch.getLength() >= 0) { + urlPrefix.append("&offset=").append(fetch.getOffset()).append("&length=").append(fetch.getLength()); + } + List fetchURLs = new ArrayList(); if(includeParts) { if (fetch.getType() == HASH_SHUFFLE || fetch.getType() == SCATTERED_HASH_SHUFFLE) { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java index e60f6935c4..7335d1cd10 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java @@ -51,10 +51,12 @@ import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.planner.logical.*; import org.apache.tajo.ipc.TajoMasterProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto; import org.apache.tajo.master.*; import org.apache.tajo.master.TaskRunnerGroupEvent.EventType; import org.apache.tajo.master.event.*; import org.apache.tajo.master.event.QueryUnitAttemptScheduleEvent.QueryUnitAttemptScheduleContext; +import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry; import org.apache.tajo.storage.AbstractStorageManager; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.KeyValueSet; @@ -252,6 +254,7 @@ public class SubQuery implements EventHandler { private int killedObjectCount = 0; private int failedObjectCount = 0; private TaskSchedulerContext schedulerContext; + private List hashShuffleIntermediateEntries = new ArrayList(); public SubQuery(QueryMasterTask.QueryMasterTaskContext context, MasterPlan masterPlan, ExecutionBlock block, AbstractStorageManager sm) { this.context = context; @@ -1096,24 +1099,37 @@ public void transition(SubQuery subQuery, SubQueryEvent subQueryEvent) { } private void cleanup() { - stopScheduler(); - releaseContainers(); - - if (!getContext().getConf().getBoolVar(TajoConf.ConfVars.TAJO_DEBUG)) { - List childs = getMasterPlan().getChilds(getId()); - List ebIds = Lists.newArrayList(); - for (ExecutionBlock executionBlock : childs){ - ebIds.add(executionBlock.getId().getProto()); - } + try { + stopScheduler(); + releaseContainers(); - try { - getContext().getQueryMasterContext().getQueryMaster().cleanupExecutionBlock(ebIds); - } catch (Throwable e) { - LOG.error(e); - } + //if (!getContext().getConf().getBoolVar(TajoConf.ConfVars.TAJO_DEBUG)) { + List childs = getMasterPlan().getChilds(getId()); + List ebIds = Lists.newArrayList(); + // first is current ebid + ebIds.add(getId().getProto()); + + for (ExecutionBlock executionBlock : childs) { + ebIds.add(executionBlock.getId().getProto()); + } + + List intermEntries = + getContext().getQueryMasterContext().getQueryMaster().cleanupExecutionBlock(this, ebIds); + + for (IntermediateEntryProto entry : intermEntries) { + hashShuffleIntermediateEntries.add(new IntermediateEntry(entry)); + } + //} + } catch (Throwable e) { + //TODO processing error + LOG.error(e.getMessage(), e); } } + public List getHashShuffleIntermediateEntries() { + return hashShuffleIntermediateEntries; + } + private static class SubQueryCompleteTransition implements MultipleArcTransition { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java index 869c1066d0..123af4804d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java @@ -48,6 +48,9 @@ public class FetchImpl implements ProtoObject, Cl private List taskIds; // repeated, the task ids private List attemptIds; // repeated, the attempt ids + private long offset = -1; + private long length = -1; + public FetchImpl() { builder = TajoWorkerProtocol.FetchProto.newBuilder(); taskIds = new ArrayList(); @@ -63,6 +66,14 @@ public FetchImpl(TajoWorkerProtocol.FetchProto proto) { proto.getHasNext(), proto.getName(), proto.getTaskIdList(), proto.getAttemptIdList()); + + if (proto.hasOffset()) { + this.offset = proto.getOffset(); + } + + if (proto.hasLength()) { + this.length = proto.getLength(); + } } public FetchImpl(QueryUnit.PullHost host, TajoWorkerProtocol.ShuffleType type, ExecutionBlockId executionBlockId, @@ -119,6 +130,9 @@ public TajoWorkerProtocol.FetchProto getProto() { Preconditions.checkArgument(taskIds.size() == attemptIds.size()); builder.addAllTaskId(taskIds); builder.addAllAttemptId(attemptIds); + + builder.setOffset(offset); + builder.setLength(length); return builder.build(); } @@ -201,6 +215,22 @@ public List getAttemptIds() { return attemptIds; } + public long getOffset() { + return offset; + } + + public void setOffset(long offset) { + this.offset = offset; + } + + public long getLength() { + return length; + } + + public void setLength(long length) { + this.length = length; + } + public FetchImpl clone() throws CloneNotSupportedException { FetchImpl newFetchImpl = (FetchImpl) super.clone(); @@ -218,6 +248,8 @@ public FetchImpl clone() throws CloneNotSupportedException { if (attemptIds != null) { newFetchImpl.attemptIds = Lists.newArrayList(attemptIds); } + newFetchImpl.offset = offset; + newFetchImpl.length = length; return newFetchImpl; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index deb2885d3b..126340ef24 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.shell.PathData; import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.util.RackResolver; +import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.QueryId; import org.apache.tajo.TajoConstants; import org.apache.tajo.TajoProtos; @@ -35,6 +36,10 @@ import org.apache.tajo.catalog.CatalogService; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoMasterProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol.ExecutionBlockReport; +import org.apache.tajo.ipc.TajoWorkerProtocol.FailureIntermediateProto; +import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto; +import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto.PageProto; import org.apache.tajo.master.querymaster.QueryMaster; import org.apache.tajo.master.querymaster.QueryMasterManagerService; import org.apache.tajo.master.rm.TajoWorkerResourceManager; @@ -43,10 +48,8 @@ import org.apache.tajo.rpc.RpcConnectionPool; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.storage.HashShuffleAppenderManager; -import org.apache.tajo.util.CommonTestingUtil; -import org.apache.tajo.util.NetUtils; -import org.apache.tajo.util.StringUtils; -import org.apache.tajo.util.TajoIdUtils; +import org.apache.tajo.storage.HashShuffleAppenderManager.HashShuffleIntermediate; +import org.apache.tajo.util.*; import org.apache.tajo.util.metrics.TajoSystemMetrics; import org.apache.tajo.webapp.StaticHttpServer; @@ -485,6 +488,63 @@ public TajoSystemMetrics getWorkerSystemMetrics() { public HashShuffleAppenderManager getHashShuffleAppenderManager() { return hashShuffleAppenderManager; } + + public ExecutionBlockReport closeHashShuffle(ExecutionBlockId ebId) { + ExecutionBlockReport.Builder reporterBuilder = ExecutionBlockReport.newBuilder(); + reporterBuilder.setEbId(ebId.getProto()); + reporterBuilder.setReportSuccess(true); + try { + List intermediateEntries = new ArrayList(); + List shuffles = hashShuffleAppenderManager.close(ebId); + if (shuffles == null) { + reporterBuilder.addAllIntermediateEntries(intermediateEntries); + return reporterBuilder.build(); + } + + IntermediateEntryProto.Builder intermediateBuilder = IntermediateEntryProto.newBuilder(); + PageProto.Builder pageBuilder = PageProto.newBuilder(); + FailureIntermediateProto.Builder failureBuilder = FailureIntermediateProto.newBuilder(); + + for (HashShuffleIntermediate eachShuffle: shuffles) { + List pages = new ArrayList(); + List failureIntermediateItems = new ArrayList(); + + for (Pair eachPage: eachShuffle.getPages()) { + pageBuilder.clear(); + pageBuilder.setPos(eachPage.getFirst()); + pageBuilder.setLength(eachPage.getSecond()); + pages.add(pageBuilder.build()); + } + + for(Pair> eachFailure: eachShuffle.getFailureTskTupleIndexes()) { + failureBuilder.clear(); + failureBuilder.setPagePos(eachFailure.getFirst()); + failureBuilder.setStartRowNum(eachFailure.getSecond().getFirst()); + failureBuilder.setEndRowNum(eachFailure.getSecond().getSecond()); + failureIntermediateItems.add(failureBuilder.build()); + } + intermediateBuilder.clear(); + intermediateBuilder.setEbId(ebId.getProto()) + .setHost(workerContext.getTajoWorkerManagerService().getBindAddr().getHostName() + ":" + getPullService().getPort()) + .setTaskId(-1) + .setAttemptId(-1) + .setPartId(eachShuffle.getPartId()) + .setVolume(eachShuffle.getVolume()) + .addAllPages(pages) + .addAllFailures(failureIntermediateItems); + intermediateEntries.add(intermediateBuilder.build()); + } + + // send intermediateEntries to QueryMaster + reporterBuilder.addAllIntermediateEntries(intermediateEntries); + + } catch (Exception e) { + LOG.error(e.getMessage(), e); + reporterBuilder.setReportSuccess(false); + reporterBuilder.setReportErrorMessage(e.getMessage()); + } + return reporterBuilder.build(); + } } public void stopWorkerForce() { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java index e77da70122..04f0165c8d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java @@ -31,6 +31,7 @@ import org.apache.tajo.TajoIdProtos; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol.ExecutionBlockReport; import org.apache.tajo.rpc.AsyncRpcServer; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.util.NetUtils; @@ -154,14 +155,20 @@ public void cleanup(RpcController controller, TajoIdProtos.QueryIdProto request, } @Override - public void cleanupExecutionBlocks(RpcController controller, TajoWorkerProtocol.ExecutionBlockListProto request, - RpcCallback done) { - for (TajoIdProtos.ExecutionBlockIdProto executionBlockIdProto : request.getExecutionBlockIdList()) { + public void cleanupExecutionBlocks(RpcController controller, + TajoWorkerProtocol.ExecutionBlockListProto ebIds, + RpcCallback done) { + ExecutionBlockReport report = null; + for (TajoIdProtos.ExecutionBlockIdProto executionBlockIdProto : ebIds.getExecutionBlockIdList()) { + if (report == null) { + report = workerContext.closeHashShuffle(new ExecutionBlockId(executionBlockIdProto)); + continue; + } String inputDir = TaskRunner.getBaseInputDir(new ExecutionBlockId(executionBlockIdProto)).toString(); workerContext.cleanup(inputDir); String outputDir = TaskRunner.getBaseOutputDir(new ExecutionBlockId(executionBlockIdProto)).toString(); workerContext.cleanup(outputDir); } - done.run(TajoWorker.TRUE_PROTO); + done.run(report); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index a6302bcc49..cd7cd1a7cf 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -436,6 +436,7 @@ public void run() { } finally { context.setProgress(1.0f); taskRunnerContext.completedTasksNum.incrementAndGet(); + context.getHashShuffleAppenderManager().taskFinished(taskId); if (killed || aborted) { context.setExecutorProgress(0.0f); @@ -485,7 +486,6 @@ public void run() { masterProxy.done(null, report, NullCallback.get()); taskRunnerContext.succeededTasksNum.incrementAndGet(); } - finishTime = System.currentTimeMillis(); LOG.info(context.getTaskId() + " completed. " + "Worker's task counter - total:" + taskRunnerContext.completedTasksNum.intValue() + diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java index 6e5c4fd144..24470efc5b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java @@ -23,13 +23,11 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.CompositeService; -import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.QueryUnitAttemptId; import org.apache.tajo.conf.TajoConf; import java.util.*; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; public class TaskRunnerManager extends CompositeService { private static final Log LOG = LogFactory.getLog(TaskRunnerManager.class); @@ -92,17 +90,17 @@ public void stopTask(String id) { synchronized(taskRunnerMap) { TaskRunner taskRunner = taskRunnerMap.remove(id); if (taskRunner != null) { - synchronized(taskRunnerCompleteCounter) { - ExecutionBlockId ebId = taskRunner.getContext().getExecutionBlockId(); - AtomicInteger counter = taskRunnerCompleteCounter.get(ebId); - if (counter != null) { - if (counter.decrementAndGet() <= 0) { - LOG.info(ebId + "'s all tasks are completed."); - workerContext.getHashShuffleAppenderManager().close(ebId); - taskRunnerCompleteCounter.remove(ebId); - } - } - } +// synchronized(taskRunnerCompleteCounter) { +// ExecutionBlockId ebId = taskRunner.getContext().getExecutionBlockId(); +// Pair counter = taskRunnerCompleteCounter.get(ebId); +// if (counter != null) { +// if (counter.getSecond().decrementAndGet() <= 0) { +// LOG.info(ebId + "'s all tasks are completed."); +// closeHashShuffle(ebId, counter.getFirst().get()); +// taskRunnerCompleteCounter.remove(ebId); +// } +// } +// } } } if(workerContext.isYarnContainerMode()) { @@ -161,7 +159,10 @@ public int getNumTasks() { } } - Map taskRunnerCompleteCounter = new HashMap(); + // <#t asks, # running tasks> +// Map> taskRunnerCompleteCounter = +// new HashMap>(); + public void startTask(final String[] params) { //TODO change to use event dispatcher Thread t = new Thread() { @@ -178,16 +179,16 @@ public void run() { taskRunnerHistoryMap.put(taskRunner.getId(), taskRunner.getContext().getExcutionBlockHistory()); } - synchronized(taskRunnerCompleteCounter) { - ExecutionBlockId ebId = taskRunner.getContext().getExecutionBlockId(); - AtomicInteger counter = taskRunnerCompleteCounter.get(ebId); - if (counter == null) { - counter = new AtomicInteger(0); - taskRunnerCompleteCounter.put(ebId, counter); - } - counter.incrementAndGet(); - - } +// synchronized(taskRunnerCompleteCounter) { +// ExecutionBlockId ebId = taskRunner.getContext().getExecutionBlockId(); +// Pair counter = taskRunnerCompleteCounter.get(ebId); +// if (counter == null) { +// counter = new Pair(new AtomicInteger(0), new AtomicInteger(0)); +// taskRunnerCompleteCounter.put(ebId, counter); +// } +// counter.getFirst().incrementAndGet(); +// counter.getSecond().incrementAndGet(); +// } taskRunner.init(systemConf); taskRunner.start(); } catch (Exception e) { diff --git a/tajo-core/src/main/proto/QueryMasterProtocol.proto b/tajo-core/src/main/proto/QueryMasterProtocol.proto index e12c9aa7c5..ebc54a4f9c 100644 --- a/tajo-core/src/main/proto/QueryMasterProtocol.proto +++ b/tajo-core/src/main/proto/QueryMasterProtocol.proto @@ -35,6 +35,7 @@ service QueryMasterProtocolService { rpc ping (QueryUnitAttemptIdProto) returns (BoolProto); rpc fatalError(TaskFatalErrorReport) returns (BoolProto); rpc done (TaskCompletionReport) returns (BoolProto); + //rpc doneExecutionBlock(ExecutionBlockCompeletReporter) returns (BoolProto); //from TajoMaster's QueryJobManager rpc killQuery(QueryIdProto) returns (BoolProto); diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto index dc2b1d7623..b6638aacb8 100644 --- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto +++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto @@ -88,9 +88,38 @@ message FetchProto { //repeated part repeated int32 taskId = 9 [packed=true]; repeated int32 attemptId = 10 [packed=true]; + + optional int64 offset = 11; + optional int64 length = 12; } +message FailureIntermediateProto { + required int64 pagePos = 1; + required int32 startRowNum = 2; + required int32 endRowNum = 3; +} +message IntermediateEntryProto { + message PageProto { + required int64 pos = 1; + required int32 length = 2; + } + required ExecutionBlockIdProto ebId = 1; + required int32 taskId = 2; + required int32 attemptId = 3; + required int32 partId = 4; + required string host = 5; + required int64 volume = 6; + repeated PageProto pages = 7; + repeated FailureIntermediateProto failures = 8; +} + +message ExecutionBlockReport { + required ExecutionBlockIdProto ebId = 1; + required bool reportSuccess = 2; + optional string reportErrorMessage = 3; + repeated IntermediateEntryProto intermediateEntries = 4; +} message QueryUnitResponseProto { required string id = 1; @@ -190,7 +219,7 @@ service TajoWorkerProtocolService { rpc executeExecutionBlock(RunExecutionBlockRequestProto) returns (BoolProto); rpc killTaskAttempt(QueryUnitAttemptIdProto) returns (BoolProto); rpc cleanup(QueryIdProto) returns (BoolProto); - rpc cleanupExecutionBlocks(ExecutionBlockListProto) returns (BoolProto); + rpc cleanupExecutionBlocks(ExecutionBlockListProto) returns (ExecutionBlockReport); } message EnforceProperty { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java index 79efd92ce8..54801087d3 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java @@ -615,12 +615,8 @@ public int compare(QueryMasterTask o1, QueryMasterTask o2) { assertTrue(!subQueries.isEmpty()); for (SubQuery subQuery: subQueries) { if (subQuery.getId().toStringNoPrefix().endsWith("_000001")) { - QueryUnit[] queryUnits = subQuery.getQueryUnits(); - assertNotNull(queryUnits); - for (QueryUnit eachQueryUnit: queryUnits) { - for (ShuffleFileOutput output: eachQueryUnit.getShuffleFileOutputs()) { - partitionIds.add(output.getPartId()); - } + for (QueryUnit.IntermediateEntry eachInterm: subQuery.getHashShuffleIntermediateEntries()) { + partitionIds.add(eachInterm.getPartId()); } } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java index fbd1ae35f8..b59c49515b 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java @@ -90,9 +90,9 @@ public TestJoinQuery(String joinOption) { public static Collection generateParameters() { return Arrays.asList(new Object[][]{ {"Hash_NoBroadcast"}, -// {"Sort_NoBroadcast"}, -// {"Hash"}, -// {"Sort"}, + {"Sort_NoBroadcast"}, + {"Hash"}, + {"Sort"}, }); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java index d9aea53974..9e3936e480 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java @@ -31,25 +31,33 @@ import org.apache.tajo.TajoTestingCluster; import org.apache.tajo.catalog.CatalogService; import org.apache.tajo.catalog.CatalogUtil; +import org.apache.tajo.catalog.Schema; import org.apache.tajo.catalog.TableDesc; +import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.planner.global.DataChannel; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.planner.logical.NodeType; import org.apache.tajo.ipc.ClientProtos; import org.apache.tajo.jdbc.TajoResultSet; +import org.apache.tajo.master.querymaster.Query; import org.apache.tajo.master.querymaster.QueryMasterTask; +import org.apache.tajo.master.querymaster.QueryUnit; +import org.apache.tajo.master.querymaster.SubQuery; +import org.apache.tajo.storage.StorageConstants; +import org.apache.tajo.util.KeyValueSet; import org.apache.tajo.worker.TajoWorker; import org.junit.Test; import java.io.IOException; import java.sql.ResultSet; -import java.util.List; -import java.util.Map; +import java.util.*; import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleType.SCATTERED_HASH_SHUFFLE; import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; public class TestTablePartitions extends QueryTestCaseBase { @@ -786,4 +794,65 @@ private MasterPlan getQueryPlan(ResultSet res) { fail("Can't find query from workers" + queryId); return null; } + + @Test + public void testScatteredHashShuffle() throws Exception { + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.DIST_QUERY_TABLE_PARTITION_VOLUME.varname, "2"); + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME.varname, "1"); + try { + KeyValueSet tableOptions = new KeyValueSet(); + tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER); + tableOptions.put(StorageConstants.CSVFILE_NULL, "\\\\N"); + + Schema schema = new Schema(); + schema.addColumn("col1", TajoDataTypes.Type.TEXT); + schema.addColumn("col2", TajoDataTypes.Type.TEXT); + + List data = new ArrayList(); + int totalBytes = 0; + Random rand = new Random(System.currentTimeMillis()); + String col2Data = "Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2" + + "Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2" + + "Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2"; + + int index = 0; + while(true) { + int col1RandomValue = 1; + String str = col1RandomValue + "|col2-" + index + "-" + col2Data; + data.add(str); + + totalBytes += str.getBytes().length; + + if (totalBytes > 4 * 1024 * 1024) { + break; + } + index++; + } + + TajoTestingCluster.createTable("testscatteredhashshuffle", schema, tableOptions, data.toArray(new String[]{}), 3); + CatalogService catalog = testingCluster.getMaster().getCatalog(); + assertTrue(catalog.existsTable("default", "testscatteredhashshuffle")); + + executeString("create table test_partition (col2 text) partition by column (col1 text)").close(); + executeString("insert into test_partition select col2, col1 from testscatteredhashshuffle").close(); + + ResultSet res = executeString("select col1 from test_partition"); + + int numRows = 0; + while (res.next()) { + numRows++; + } + assertEquals(data.size(), numRows); + + // assert data file size + + } finally { + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.DIST_QUERY_TABLE_PARTITION_VOLUME.varname, + TajoConf.ConfVars.DIST_QUERY_TABLE_PARTITION_VOLUME.defaultVal); + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME.varname, + TajoConf.ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME.defaultVal); + executeString("DROP TABLE test_partition PURGE").close(); + executeString("DROP TABLE testScatteredHashShuffle PURGE").close(); + } + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java b/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java index 33d2454ec8..2909fd5d04 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java @@ -24,14 +24,15 @@ import org.apache.tajo.TestTajoIds; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.querymaster.QueryUnit; +import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry; import org.apache.tajo.master.querymaster.Repartitioner; import org.apache.tajo.util.Pair; -import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.FetchImpl; +import org.jboss.netty.handler.codec.http.QueryStringDecoder; import org.junit.Test; -import java.util.List; -import java.util.Map; +import java.net.URI; +import java.util.*; import static junit.framework.Assert.assertEquals; import static org.apache.tajo.master.querymaster.Repartitioner.FetchGroupMeta; @@ -44,33 +45,63 @@ public void testCreateHashFetchURL() throws Exception { String hostName = "tajo1"; int port = 1234; ExecutionBlockId sid = new ExecutionBlockId(q1, 2); - int partitionId = 2; + int numPartition = 10; - List intermediateEntries = TUtil.newList(); + Map> intermediateEntries = new HashMap>(); + for (int i = 0; i < numPartition; i++) { + intermediateEntries.put(i, new ArrayList()); + } for (int i = 0; i < 1000; i++) { - intermediateEntries.add(new QueryUnit.IntermediateEntry(i, 0, partitionId, new QueryUnit.PullHost(hostName, port))); + int partitionId = i % numPartition; + IntermediateEntry entry = new IntermediateEntry(i, 0, partitionId, new QueryUnit.PullHost(hostName, port)); + entry.setEbId(sid); + entry.setVolume(10); + intermediateEntries.get(partitionId).add(entry); } - FetchImpl fetch = new FetchImpl(new QueryUnit.PullHost(hostName, port), TajoWorkerProtocol.ShuffleType.HASH_SHUFFLE, - sid, partitionId, intermediateEntries); + Map>> hashEntries = + new HashMap>>(); - fetch.setName(sid.toString()); + for (Map.Entry> eachEntry: intermediateEntries.entrySet()) { + FetchImpl fetch = new FetchImpl(new QueryUnit.PullHost(hostName, port), TajoWorkerProtocol.ShuffleType.HASH_SHUFFLE, + sid, eachEntry.getKey(), eachEntry.getValue()); - TajoWorkerProtocol.FetchProto proto = fetch.getProto(); - fetch = new FetchImpl(proto); - assertEquals(proto, fetch.getProto()); - } + fetch.setName(sid.toString()); + + TajoWorkerProtocol.FetchProto proto = fetch.getProto(); + fetch = new FetchImpl(proto); + assertEquals(proto, fetch.getProto()); + + Map> ebEntries = new HashMap>(); + ebEntries.put(sid, eachEntry.getValue()); + + hashEntries.put(eachEntry.getKey(), ebEntries); + + List uris = fetch.getURIs(); + assertEquals(1, uris.size()); //In Hash Suffle, Fetcher return only one URI per partition. -// private List splitMaps(List mapq) { -// if (null == mapq) { -// return null; -// } -// final List ret = new ArrayList(); -// for (String s : mapq) { -// Collections.addAll(ret, s.split(",")); -// } -// return ret; -// } + URI uri = uris.get(0); + final Map> params = + new QueryStringDecoder(uri).getParameters(); + + assertEquals(eachEntry.getKey().toString(), params.get("p").get(0)); + assertEquals("h", params.get("type").get(0)); + assertEquals("" + sid.getId(), params.get("sid").get(0)); + } + + Map>> mergedHashEntries = + Repartitioner.mergeIntermediateByPullHost(hashEntries); + + assertEquals(numPartition, mergedHashEntries.size()); + for (int i = 0; i < numPartition; i++) { + Map> eachEntry = mergedHashEntries.get(0); + assertEquals(1, eachEntry.size()); + List interEntry = eachEntry.get(sid); + assertEquals(1, interEntry.size()); + + assertEquals(1000, interEntry.get(0).getVolume()); + } + } @Test public void testScheduleFetchesByEvenDistributedVolumes() { @@ -119,4 +150,242 @@ private static void assertFetchVolumes(long [] expected, Long [] results) { assertTrue(expected[i] + " is expected, but " + results[i], expected[i] == results[i]); } } + + @Test + public void testMergeIntermediates() { + //Test Merge + List intermediateEntries = new ArrayList(); + + int[] pageLengths = {10 * 1024 * 1024, 10 * 1024 * 1024, 10 * 1024 * 1024, 5 * 1024 * 1024}; //35 MB + long expectedTotalLength = 0; + for (int i = 0; i < 20; i++) { + List> pages = new ArrayList>(); + long offset = 0; + for (int j = 0; j < pageLengths.length; j++) { + pages.add(new Pair(offset, pageLengths[j])); + offset += pageLengths[j]; + expectedTotalLength += pageLengths[j]; + } + IntermediateEntry interm = new IntermediateEntry(i, -1, -1, new QueryUnit.PullHost("" + i, i)); + interm.setPages(pages); + interm.setVolume(offset); + intermediateEntries.add(interm); + } + + long splitVolume = 128 * 1024 * 1024; + List> fetches = Repartitioner.splitOrMergeIntermediates(null, intermediateEntries, + splitVolume, 10 * 1024 * 1024); + assertEquals(6, fetches.size()); + + int totalInterms = 0; + int index = 0; + int numZeroPosFetcher = 0; + long totalLength = 0; + for (List eachFetchList: fetches) { + totalInterms += eachFetchList.size(); + long eachFetchVolume = 0; + for (FetchImpl eachFetch: eachFetchList) { + eachFetchVolume += eachFetch.getLength(); + if (eachFetch.getOffset() == 0) { + numZeroPosFetcher++; + } + totalLength += eachFetch.getLength(); + } + assertTrue(eachFetchVolume + " should be smaller than splitVolume", eachFetchVolume < splitVolume); + if (index < fetches.size() - 1) { + assertTrue(eachFetchVolume + " should be great than 100MB", eachFetchVolume >= 100 * 1024 * 1024); + } + index++; + } + assertEquals(23, totalInterms); + assertEquals(20, numZeroPosFetcher); + assertEquals(expectedTotalLength, totalLength); + } + + @Test + public void testSplitIntermediates() { + List intermediateEntries = new ArrayList(); + + int[] pageLengths = new int[20]; //195MB + for (int i = 0 ; i < pageLengths.length; i++) { + if (i < pageLengths.length - 1) { + pageLengths[i] = 10 * 1024 * 1024; + } else { + pageLengths[i] = 5 * 1024 * 1024; + } + } + + long expectedTotalLength = 0; + for (int i = 0; i < 20; i++) { + List> pages = new ArrayList>(); + long offset = 0; + for (int j = 0; j < pageLengths.length; j++) { + pages.add(new Pair(offset, pageLengths[j])); + offset += pageLengths[j]; + expectedTotalLength += pageLengths[j]; + } + IntermediateEntry interm = new IntermediateEntry(i, -1, 0, new QueryUnit.PullHost("" + i, i)); + interm.setPages(pages); + interm.setVolume(offset); + intermediateEntries.add(interm); + } + + long splitVolume = 128 * 1024 * 1024; + List> fetches = Repartitioner.splitOrMergeIntermediates(null, intermediateEntries, + splitVolume, 10 * 1024 * 1024); + assertEquals(32, fetches.size()); + + int index = 0; + int numZeroPosFetcher = 0; + long totalLength = 0; + Set uniqPullHost = new HashSet(); + + for (List eachFetchList: fetches) { + long length = 0; + for (FetchImpl eachFetch: eachFetchList) { + if (eachFetch.getOffset() == 0) { + numZeroPosFetcher++; + } + totalLength += eachFetch.getLength(); + length += eachFetch.getLength(); + uniqPullHost.add(eachFetch.getPullHost().toString()); + } + assertTrue(length + " should be smaller than splitVolume", length < splitVolume); + if (index < fetches.size() - 1) { + assertTrue(length + " should be great than 100MB" + fetches.size() + "," + index, length >= 100 * 1024 * 1024); + } + index++; + } + assertEquals(20, numZeroPosFetcher); + assertEquals(20, uniqPullHost.size()); + assertEquals(expectedTotalLength, totalLength); + } + + @Test + public void testSplitIntermediates2() { + long[][] pageDatas = { + {0, 10538717}, + {10538717, 10515884}, + {21054601, 10514343}, + {31568944, 10493988}, + {42062932, 10560639}, + {52623571, 10548486}, + {63172057, 10537811}, + {73709868, 10571060}, + {84280928, 10515062}, + {94795990, 10502964}, + {105298954, 10514011}, + {115812965, 10532154}, + {126345119, 10534133}, + {136879252, 10549749}, + {147429001, 10566547}, + {157995548, 10543700}, + {168539248, 10490324}, + {179029572, 10500720}, + {189530292, 10505425}, + {200035717, 10548418}, + {210584135, 10562887}, + {221147022, 10554967}, + {231701989, 10507297}, + {242209286, 10515612}, + {252724898, 10491274}, + {263216172, 10512956}, + {273729128, 10490736}, + {284219864, 10501878}, + {294721742, 10564568}, + {305286310, 10488896}, + {315775206, 10516308}, + {326291514, 10517965}, + {336809479, 10487038}, + {347296517, 10603472}, + {357899989, 10507330}, + {368407319, 10549429}, + {378956748, 10533443}, + {389490191, 10530852}, + {400021043, 11036431}, + {411057474, 10541007}, + {421598481, 10600477}, + {432198958, 10519805}, + {442718763, 10500769}, + {453219532, 10507192}, + {463726724, 10540424}, + {474267148, 10509129}, + {484776277, 10527100}, + {495303377, 10720789}, + {506024166, 10568542}, + {516592708, 11046886}, + {527639594, 10580358}, + {538219952, 10508940}, + {548728892, 10523968}, + {559252860, 10580626}, + {569833486, 10539361}, + {580372847, 10496662}, + {590869509, 10505280}, + {601374789, 10564655}, + {611939444, 10505842}, + {622445286, 10523889}, + {632969175, 10553186}, + {643522361, 10535866}, + {654058227, 10501796}, + {664560023, 10530358}, + {675090381, 10585340}, + {685675721, 10602017}, + {696277738, 10546614}, + {706824352, 10511511}, + {717335863, 11019221}, + {728355084, 10558143}, + {738913227, 10516245}, + {749429472, 10502613}, + {759932085, 10522145}, + {770454230, 10489373}, + {780943603, 10520973}, + {791464576, 11021218}, + {802485794, 10496362}, + {812982156, 10502354}, + {823484510, 10515932}, + {834000442, 10591044}, + {844591486, 5523957} + }; + + List entries = new ArrayList(); + for (int i = 0; i < 2; i++) { + List> pages = new ArrayList>(); + for (int j = 0; j < pageDatas.length; j++) { + pages.add(new Pair(pageDatas[j][0], (int) (pageDatas[j][1]))); + } + IntermediateEntry entry = new IntermediateEntry(-1, -1, 1, new QueryUnit.PullHost("host" + i , 9000)); + entry.setPages(pages); + + entries.add(entry); + } + + long splitVolume = 256 * 1024 * 1024; + List> fetches = Repartitioner.splitOrMergeIntermediates(null, entries, splitVolume, + 10 * 1024 * 1024); + + + long[][] expected = { + {0,263216172}, + {263216172,264423422}, + {527639594,263824982}, + {791464576,58650867}, + {0,200035717}, + {200035717,263691007}, + {463726724,264628360}, + {728355084,121760359}, + }; + int index = 0; + for (List eachFetchList: fetches) { + if (index == 3) { + assertEquals(2, eachFetchList.size()); + } else { + assertEquals(1, eachFetchList.size()); + } + for (FetchImpl eachFetch: eachFetchList) { + assertEquals(expected[index][0], eachFetch.getOffset()); + assertEquals(expected[index][1], eachFetch.getLength()); + index++; + } + } + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestIntermediateEntry.java b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestIntermediateEntry.java new file mode 100644 index 0000000000..114b232512 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestIntermediateEntry.java @@ -0,0 +1,53 @@ +/** + * 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.tajo.master.querymaster; + +import org.apache.tajo.util.Pair; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class TestIntermediateEntry { + @Test + public void testPage() { + QueryUnit.IntermediateEntry interm = new QueryUnit.IntermediateEntry(-1, -1, 1, null); + + List> pages = new ArrayList>(); + pages.add(new Pair(0L, 1441275)); + pages.add(new Pair(1441275L, 1447446)); + pages.add(new Pair(2888721L, 1442507)); + + interm.setPages(pages); + + long splitBytes = 3 * 1024 * 1024; + + List> splits = interm.split(splitBytes, splitBytes); + assertEquals(2, splits.size()); + + long[][] expected = { {0, 1441275 + 1447446}, {1441275 + 1447446, 1442507} }; + for (int i = 0; i < 2; i++) { + Pair eachSplit = splits.get(i); + assertEquals(expected[i][0], eachSplit.getFirst().longValue()); + assertEquals(expected[i][1], eachSplit.getSecond().longValue()); + } + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java index f98873517e..15dba34fde 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java @@ -56,7 +56,7 @@ public final void case1() throws Exception { // tpch/lineitem.tbl long[] expectedNumRows = new long[]{5, 2, 2, 2}; - long[] expectedNumBytes = new long[]{604, 18, 18, 8}; + long[] expectedNumBytes = new long[]{604, 604, 18, 8}; long[] expectedReadBytes = new long[]{604, 18, 18, 0}; assertStatus(2, expectedNumRows, expectedNumBytes, expectedReadBytes); @@ -74,7 +74,7 @@ public final void case2() throws Exception { // tpch/lineitem.tbl long[] expectedNumRows = new long[]{5, 2, 2, 2, 2, 2}; - long[] expectedNumBytes = new long[]{604, 162, 162, 138, 138, 194}; + long[] expectedNumBytes = new long[]{604, 604, 162, 138, 138, 194}; long[] expectedReadBytes = new long[]{604, 162, 162, 0, 138, 0}; assertStatus(3, expectedNumRows, expectedNumBytes, expectedReadBytes); @@ -105,7 +105,7 @@ public final void case3() throws Exception { // in/out * subquery(4) long[] expectedNumRows = new long[]{2, 2, 5, 5, 7, 2, 2, 2}; - long[] expectedNumBytes = new long[]{8, 34, 20, 75, 109, 34, 34, 18}; + long[] expectedNumBytes = new long[]{8, 8, 20, 20, 109, 34, 34, 18}; long[] expectedReadBytes = new long[]{8, 34, 20, 75, 109, 0, 34, 0}; assertStatus(4, expectedNumRows, expectedNumBytes, expectedReadBytes); diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java index de6eb62ff7..c4b89c1b72 100644 --- a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java +++ b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java @@ -18,29 +18,70 @@ package org.apache.tajo.storage; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.QueryUnitAttemptId; import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.util.Pair; import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; public class HashShuffleAppender implements Appender { + private static Log LOG = LogFactory.getLog(HashShuffleAppender.class); + private FileAppender appender; private AtomicBoolean closed = new AtomicBoolean(false); private int partId; private TableStats tableStats; - public HashShuffleAppender(int partId, FileAppender appender) { + //>> + private Map>>> taskTupleIndexes; + + //page start offset, length + private List> pages = new ArrayList>(); + + private Pair currentPage; + + private int pageSize; //MB + + private int rowNumInPage; + + private int totalRows; + + private long offset; + + private ExecutionBlockId ebId; + + public HashShuffleAppender(ExecutionBlockId ebId, int partId, int pageSize, FileAppender appender) { + this.ebId = ebId; this.partId = partId; this.appender = appender; + this.pageSize = pageSize; } @Override public void init() throws IOException { + currentPage = new Pair(0L, 0); + taskTupleIndexes = new HashMap>>>(); + rowNumInPage = 0; } - public long addTuples(List tuples) throws IOException { + /** + * Write multiple tuples. Each tuple is written by a FileAppender which is responsible specified partition. + * After writing if a current page exceeds pageSize, pageOffset will be added. + * @param taskId + * @param tuples + * @return written bytes + * @throws IOException + */ + public int addTuples(QueryUnitAttemptId taskId, List tuples) throws IOException { synchronized(appender) { if (closed.get()) { return 0; @@ -50,18 +91,47 @@ public long addTuples(List tuples) throws IOException { for (Tuple eachTuple: tuples) { appender.addTuple(eachTuple); } - return appender.getOffset() - currentPos; + long posAfterWritten = appender.getOffset(); + + int writtenBytes = (int)(posAfterWritten - currentPos); + + int nextRowNum = rowNumInPage + tuples.size(); + List>> taskIndexes = taskTupleIndexes.get(taskId); + if (taskIndexes == null) { + taskIndexes = new ArrayList>>(); + taskTupleIndexes.put(taskId, taskIndexes); + } + taskIndexes.add( + new Pair>(currentPage.getFirst(), new Pair(rowNumInPage, nextRowNum))); + rowNumInPage = nextRowNum; + + if (posAfterWritten - currentPage.getFirst() > pageSize) { + nextPage(posAfterWritten); + rowNumInPage = 0; + } + + totalRows += tuples.size(); + return writtenBytes; } } + public long getOffset() throws IOException { + if (closed.get()) { + return offset; + } else { + return appender.getOffset(); + } + } + + private void nextPage(long pos) { + currentPage.setSecond((int) (pos - currentPage.getFirst())); + pages.add(currentPage); + currentPage = new Pair(pos, 0); + } + @Override public void addTuple(Tuple t) throws IOException { - synchronized(appender) { - if (closed.get()) { - return; - } - appender.addTuple(t); - } + throw new IOException("Not support addTuple, use addTuples()"); } @Override @@ -80,7 +150,18 @@ public void close() throws IOException { if (closed.get()) { return; } + appender.flush(); + offset = appender.getOffset(); + if (offset > currentPage.getFirst()) { + nextPage(offset); + } appender.close(); + if (!pages.isEmpty()) { + LOG.info(ebId + ",partId=" + partId + " Appender closed: fileLen=" + offset + ", pages=" + pages.size() + + ", lastPage=" + pages.get(pages.size() - 1)); + } else { + LOG.info(ebId + ",partId=" + partId + " Appender closed: fileLen=" + offset + ", pages=" + pages.size()); + } closed.set(true); tableStats = appender.getStats(); } @@ -96,4 +177,26 @@ public TableStats getStats() { return appender.getStats(); } } + + public List> getPages() { + return pages; + } + + public Map>>> getTaskTupleIndexes() { + return taskTupleIndexes; + } + + public List>> getMergedTupleIndexes() { + List>> merged = new ArrayList>>(); + + for (List>> eachFailureIndex: taskTupleIndexes.values()) { + merged.addAll(eachFailureIndex); + } + + return merged; + } + + public void taskFinished(QueryUnitAttemptId taskId) { + taskTupleIndexes.remove(taskId); + } } diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java index 791d076abe..e9555dd83d 100644 --- a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java +++ b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java @@ -25,12 +25,17 @@ import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.QueryUnitAttemptId; import org.apache.tajo.catalog.Schema; import org.apache.tajo.catalog.TableMeta; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.util.Pair; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -43,6 +48,7 @@ public class HashShuffleAppenderManager { private FileSystem defaultFS; private FileSystem localFS; private LocalDirAllocator lDirAllocator; + private int pageSize; public HashShuffleAppenderManager(TajoConf systemConf) throws IOException { this.systemConf = systemConf; @@ -53,7 +59,7 @@ public HashShuffleAppenderManager(TajoConf systemConf) throws IOException { // initialize DFS and LocalFileSystems defaultFS = TajoConf.getTajoRootDir(systemConf).getFileSystem(systemConf); localFS = FileSystem.getLocal(systemConf); - + pageSize = systemConf.getIntVar(ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME) * 1024 * 1024; } public HashShuffleAppender getAppender(TajoConf tajoConf, ExecutionBlockId ebId, int partId, @@ -78,7 +84,7 @@ public HashShuffleAppender getAppender(TajoConf tajoConf, ExecutionBlockId ebId, if (!fs.exists(dataFile.getParent())) { fs.mkdirs(dataFile.getParent()); } - FileAppender appender = (FileAppender)StorageManagerFactory.getStorageManager( + FileAppender appender = (FileAppender) StorageManagerFactory.getStorageManager( tajoConf).getAppender(meta, outSchema, dataFile); appender.enableStats(); appender.init(); @@ -86,7 +92,8 @@ public HashShuffleAppender getAppender(TajoConf tajoConf, ExecutionBlockId ebId, partitionAppenderMeta = new PartitionAppenderMeta(); partitionAppenderMeta.partId = partId; partitionAppenderMeta.dataFile = dataFile; - partitionAppenderMeta.appender = new HashShuffleAppender(partId, appender); + partitionAppenderMeta.appender = new HashShuffleAppender(ebId, partId, pageSize, appender); + partitionAppenderMeta.appender.init(); partitionAppenderMap.put(partId, partitionAppenderMeta); LOG.info("Create Hash shuffle file(partId=" + partId + "): " + dataFile); @@ -130,25 +137,89 @@ public Path getPartitionAppenderDataFile(ExecutionBlockId ebId, int partId) { return null; } - public void close(ExecutionBlockId ebId) { + public List close(ExecutionBlockId ebId) throws IOException { Map partitionAppenderMap = null; synchronized (appenderMap) { partitionAppenderMap = appenderMap.remove(ebId); } if (partitionAppenderMap == null) { - return; + LOG.info("Close HashShuffleAppender:" + ebId + ", not a hash shuffle"); + return null; } - LOG.info("Close HashShuffleAppender:" + ebId); + // Send Intermediate data to QueryMaster. + List intermEntries = new ArrayList(); for (PartitionAppenderMeta eachMeta : partitionAppenderMap.values()) { try { eachMeta.appender.close(); + HashShuffleIntermediate intermediate = + new HashShuffleIntermediate(eachMeta.partId, eachMeta.appender.getOffset(), + eachMeta.appender.getPages(), + eachMeta.appender.getMergedTupleIndexes()); + intermEntries.add(intermediate); } catch (IOException e) { + LOG.error(e.getMessage(), e); + throw e; + } + } + + LOG.info("Close HashShuffleAppender:" + ebId + ", intermediates=" + intermEntries.size()); + + return intermEntries; + } + + public void taskFinished(QueryUnitAttemptId taskId) { + synchronized (appenderMap) { + Map partitionAppenderMap = + appenderMap.get(taskId.getQueryUnitId().getExecutionBlockId()); + if (partitionAppenderMap == null) { + return; + } + + for (PartitionAppenderMeta eachAppender: partitionAppenderMap.values()) { + eachAppender.appender.taskFinished(taskId); } } } + public static class HashShuffleIntermediate { + private int partId; + + private long volume; + + //[>] + private Collection>> failureTskTupleIndexes; + + //[] + private List> pages = new ArrayList>(); + + public HashShuffleIntermediate(int partId, long volume, + List> pages, + Collection>> failureTskTupleIndexes) { + this.partId = partId; + this.volume = volume; + this.failureTskTupleIndexes = failureTskTupleIndexes; + this.pages = pages; + } + + public int getPartId() { + return partId; + } + + public long getVolume() { + return volume; + } + + public Collection>> getFailureTskTupleIndexes() { + return failureTskTupleIndexes; + } + + public List> getPages() { + return pages; + } + } + static class PartitionAppenderMeta { int partId; HashShuffleAppender appender; diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java index 97c56c803b..ceb9684ab5 100644 --- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java +++ b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java @@ -379,6 +379,8 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) final List taskIdList = params.get("ta"); final List subQueryIds = params.get("sid"); final List partIds = params.get("p"); + final List offsetList = params.get("offset"); + final List lengthList = params.get("length"); if (types == null || subQueryIds == null || qids == null || partIds == null) { sendError(ctx, "Required queryId, type, subquery Id, and part id", @@ -397,6 +399,9 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) String shuffleType = types.get(0); String sid = subQueryIds.get(0); + long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L; + long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L; + if (!shuffleType.equals("h") && !shuffleType.equals("s") && taskIdList == null) { sendError(ctx, "Required taskIds", BAD_REQUEST); } @@ -452,7 +457,17 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) Path path = localFS.makeQualified(lDirAlloc.getLocalPathToRead(partPath, conf)); File file = new File(path.toUri()); - FileChunk chunk = new FileChunk(file, 0, file.length()); + long startPos = (offset >= 0 && length >= 0) ? offset : 0; + long readLen = (offset >= 0 && length >= 0) ? length : file.length(); + + if (startPos >= file.length()) { + String errorMessage = "Start pos[" + startPos + "] great than file length [" + file.length() + "]"; + LOG.error(errorMessage); + sendError(ctx, errorMessage, BAD_REQUEST); + return; + } + LOG.info("RequestURL" + request.getUri() + ", fileLen=" + file.length()); + FileChunk chunk = new FileChunk(file, startPos, readLen); chunks.add(chunk); } else { LOG.error("Unknown shuffle type: " + shuffleType); From e02f0cdf14b502dd949cf9cc5e7c0893ec312e10 Mon Sep 17 00:00:00 2001 From: HyoungJun Kim Date: Tue, 12 Aug 2014 14:56:36 +0900 Subject: [PATCH 4/8] TAJO-992: Reduce number of hash shuffle output file. Add some debug logs --- .../physical/HashShuffleFileWriteExec.java | 2 +- .../apache/tajo/master/querymaster/Query.java | 3 +- .../tajo/master/querymaster/QueryMaster.java | 45 +----- .../QueryMasterManagerService.java | 13 ++ .../master/querymaster/Repartitioner.java | 55 ++++--- .../tajo/master/querymaster/SubQuery.java | 86 ++++++++--- .../org/apache/tajo/worker/TajoWorker.java | 81 ++-------- .../tajo/worker/TajoWorkerManagerService.java | 9 +- .../org/apache/tajo/worker/TaskRunner.java | 13 ++ .../apache/tajo/worker/TaskRunnerManager.java | 144 +++++++++++++++--- .../src/main/proto/QueryMasterProtocol.proto | 2 +- .../src/main/proto/TajoWorkerProtocol.proto | 5 +- .../pullserver/TajoPullServerService.java | 3 +- 13 files changed, 264 insertions(+), 197 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java index 281de6d5e3..cee2b77ced 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java @@ -140,7 +140,7 @@ public Tuple next() throws IOException { } TableStats aggregated = (TableStats)child.getInputStats().clone(); - aggregated.setReadBytes(writtenBytes); + aggregated.setNumBytes(writtenBytes); aggregated.setNumRows(numRows); context.setResultStats(aggregated); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java index 8bb3dde755..41b345293a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java @@ -791,12 +791,13 @@ public void transition(Query query, QueryEvent event) { if (castEvent.getState() == SubQueryState.SUCCEEDED && // latest subquery succeeded query.getState() == QueryState.QUERY_RUNNING && // current state is not in KILL_WAIT, FAILED, or ERROR. hasNext(query)) { // there remains at least one subquery. + query.getSubQuery(castEvent.getExecutionBlockId()).waitingIntermediateReport(); executeNextBlock(query); } else { // if a query is completed due to finished, kill, failure, or error query.eventHandler.handle(new QueryCompletedEvent(castEvent.getExecutionBlockId(), castEvent.getState())); } } catch (Throwable t) { - LOG.error(t); + LOG.error(t.getMessage(), t); query.eventHandler.handle(new QueryEvent(event.getQueryId(), QueryEventType.INTERNAL_ERROR)); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java index 3a6431abf4..c06951c159 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java @@ -167,9 +167,7 @@ public void stop() { } } - - protected List cleanupExecutionBlock(SubQuery subQuery, - List executionBlockIds) throws Exception { + protected void cleanupExecutionBlock(List executionBlockIds) { LOG.info("cleanup executionBlocks: " + executionBlockIds); NettyClientBase rpc = null; List workers = getAllWorker(); @@ -177,8 +175,6 @@ protected List cleanupExecutionBlock(SubQuery subQuery, builder.addAllExecutionBlockId(Lists.newArrayList(executionBlockIds)); TajoWorkerProtocol.ExecutionBlockListProto executionBlockListProto = builder.build(); - Exception err = null; - List intermediateEntries = new ArrayList(); for (TajoMasterProtocol.WorkerResourceProto worker : workers) { try { @@ -188,48 +184,13 @@ protected List cleanupExecutionBlock(SubQuery subQuery, TajoWorkerProtocol.class, true); TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub(); - CallFuture callBack = new CallFuture(); - tajoWorkerProtocolService.cleanupExecutionBlocks(null, executionBlockListProto, callBack); - long startTime = System.currentTimeMillis(); - if (err == null) { - while (true) { - try { - ExecutionBlockReport report = callBack.get(10, TimeUnit.SECONDS); - if (report == null) { - if (System.currentTimeMillis() - startTime > 180 * 1000) { - LOG.error("Can't ExecutionBlock report from " + worker.getHost() + ":" + worker.getPeerRpcPort()); - break; - } - } - if (!report.getReportSuccess()) { - throw new Exception("Getting ExecutionBlockReport from " + worker.getHost() + ":" + worker.getPeerRpcPort() + - " error: " + report.getReportErrorMessage()); - } - intermediateEntries.addAll(report.getIntermediateEntriesList()); - break; - } catch (InterruptedException e) { - LOG.error(e.getMessage(), e); - break; - } catch (TimeoutException e) { - if (System.currentTimeMillis() - startTime > 180 * 1000) { - LOG.error("Can't ExecutionBlock report from " + worker.getHost() + ":" + worker.getPeerRpcPort()); - break; - } - } - } - } + tajoWorkerProtocolService.cleanupExecutionBlocks(null, executionBlockListProto, NullCallback.get()); } catch (Exception e) { - LOG.error(e.getMessage()); - err = e; + continue; } finally { connPool.releaseConnection(rpc); } } - if (err != null) { - throw err; - } - - return intermediateEntries; } private void cleanup(QueryId queryId) { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java index f52d14348b..0ef417a2b6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java @@ -40,6 +40,8 @@ import org.apache.tajo.worker.TajoWorker; import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; public class QueryMasterManagerService extends CompositeService implements QueryMasterProtocol.QueryMasterProtocolService.Interface { @@ -215,6 +217,17 @@ public void done(RpcController controller, TajoWorkerProtocol.TaskCompletionRepo } } + @Override + public void doneExecutionBlock( + RpcController controller, TajoWorkerProtocol.ExecutionBlockReport request, + RpcCallback done) { + QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(new QueryId(request.getEbId().getQueryId())); + if (queryMasterTask != null) { + ExecutionBlockId ebId = new ExecutionBlockId(request.getEbId()); + queryMasterTask.getQuery().getSubQuery(ebId).receiveExecutionBlockReport(request); + } + } + @Override public void killQuery(RpcController controller, TajoIdProtos.QueryIdProto request, RpcCallback done) { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java index df462faebe..5e1d412640 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java @@ -321,47 +321,44 @@ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMaster scanEbId = childBlock.getId(); } SubQuery childExecSM = subQuery.getContext().getSubQuery(childBlock.getId()); - //for (QueryUnit task : childExecSM.getQueryUnits()) { - // if (task.getIntermediateData() != null && !task.getIntermediateData().isEmpty()) { - // for (IntermediateEntry intermEntry : task.getIntermediateData()) { + if (childExecSM.getHashShuffleIntermediateEntries() != null && !childExecSM.getHashShuffleIntermediateEntries().isEmpty()) { for (IntermediateEntry intermEntry: childExecSM.getHashShuffleIntermediateEntries()) { - intermEntry.setEbId(childBlock.getId()); - if (hashEntries.containsKey(intermEntry.getPartId())) { - Map> tbNameToInterm = - hashEntries.get(intermEntry.getPartId()); - - if (tbNameToInterm.containsKey(scanEbId)) { - tbNameToInterm.get(scanEbId).add(intermEntry); - } else { - tbNameToInterm.put(scanEbId, TUtil.newList(intermEntry)); - } + intermEntry.setEbId(childBlock.getId()); + if (hashEntries.containsKey(intermEntry.getPartId())) { + Map> tbNameToInterm = + hashEntries.get(intermEntry.getPartId()); + + if (tbNameToInterm.containsKey(scanEbId)) { + tbNameToInterm.get(scanEbId).add(intermEntry); } else { - Map> tbNameToInterm = - new HashMap>(); tbNameToInterm.put(scanEbId, TUtil.newList(intermEntry)); - hashEntries.put(intermEntry.getPartId(), tbNameToInterm); } - } - } else { - //if no intermidatedata(empty table), make empty entry - int emptyPartitionId = 0; - if (hashEntries.containsKey(emptyPartitionId)) { - Map> tbNameToInterm = hashEntries.get(emptyPartitionId); - if (tbNameToInterm.containsKey(scanEbId)) - tbNameToInterm.get(scanEbId).addAll(new ArrayList()); - else - tbNameToInterm.put(scanEbId, new ArrayList()); } else { Map> tbNameToInterm = new HashMap>(); - tbNameToInterm.put(scanEbId, new ArrayList()); - hashEntries.put(emptyPartitionId, tbNameToInterm); + tbNameToInterm.put(scanEbId, TUtil.newList(intermEntry)); + hashEntries.put(intermEntry.getPartId(), tbNameToInterm); } } + } else { + //if no intermidatedata(empty table), make empty entry + int emptyPartitionId = 0; + if (hashEntries.containsKey(emptyPartitionId)) { + Map> tbNameToInterm = hashEntries.get(emptyPartitionId); + if (tbNameToInterm.containsKey(scanEbId)) + tbNameToInterm.get(scanEbId).addAll(new ArrayList()); + else + tbNameToInterm.put(scanEbId, new ArrayList()); + } else { + Map> tbNameToInterm = + new HashMap>(); + tbNameToInterm.put(scanEbId, new ArrayList()); + hashEntries.put(emptyPartitionId, tbNameToInterm); + } } -// } + } // merge intermediate entry by ebid, pullhost // Map>> mergedHashEntries = mergeIntermediateByPullHost(hashEntries); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java index 7335d1cd10..e7732e7b9d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java @@ -51,6 +51,7 @@ import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.planner.logical.*; import org.apache.tajo.ipc.TajoMasterProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto; import org.apache.tajo.master.*; import org.apache.tajo.master.TaskRunnerGroupEvent.EventType; @@ -60,11 +61,13 @@ import org.apache.tajo.storage.AbstractStorageManager; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.KeyValueSet; +import org.apache.tajo.util.Pair; import org.apache.tajo.worker.FetchImpl; import java.io.IOException; import java.util.*; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -255,6 +258,7 @@ public class SubQuery implements EventHandler { private int failedObjectCount = 0; private TaskSchedulerContext schedulerContext; private List hashShuffleIntermediateEntries = new ArrayList(); + private AtomicInteger completeReportReceived = new AtomicInteger(0); public SubQuery(QueryMasterTask.QueryMasterTaskContext context, MasterPlan masterPlan, ExecutionBlock block, AbstractStorageManager sm) { this.context = context; @@ -294,6 +298,7 @@ public AbstractTaskScheduler getTaskScheduler() { } public void setStartTime() { + LOG.fatal("setStartTime>>>>>>>>>>>>>" + startTime + ">" + context.getClock().getTime()); startTime = context.getClock().getTime(); } @@ -303,6 +308,7 @@ public long getStartTime() { } public void setFinishTime() { + LOG.fatal("setFinishTime>>>>>>>>>>>>>" + startTime + ">" + context.getClock().getTime()); finishTime = context.getClock().getTime(); } @@ -1099,30 +1105,18 @@ public void transition(SubQuery subQuery, SubQueryEvent subQueryEvent) { } private void cleanup() { - try { - stopScheduler(); - releaseContainers(); - - //if (!getContext().getConf().getBoolVar(TajoConf.ConfVars.TAJO_DEBUG)) { - List childs = getMasterPlan().getChilds(getId()); - List ebIds = Lists.newArrayList(); - // first is current ebid - ebIds.add(getId().getProto()); + stopScheduler(); + releaseContainers(); - for (ExecutionBlock executionBlock : childs) { - ebIds.add(executionBlock.getId().getProto()); - } + if (!getContext().getConf().getBoolVar(TajoConf.ConfVars.TAJO_DEBUG)) { + List childs = getMasterPlan().getChilds(getId()); + List ebIds = Lists.newArrayList(); - List intermEntries = - getContext().getQueryMasterContext().getQueryMaster().cleanupExecutionBlock(this, ebIds); + for (ExecutionBlock executionBlock : childs) { + ebIds.add(executionBlock.getId().getProto()); + } - for (IntermediateEntryProto entry : intermEntries) { - hashShuffleIntermediateEntries.add(new IntermediateEntry(entry)); - } - //} - } catch (Throwable e) { - //TODO processing error - LOG.error(e.getMessage(), e); + getContext().getQueryMasterContext().getQueryMaster().cleanupExecutionBlock(ebIds); } } @@ -1130,6 +1124,56 @@ public List getHashShuffleIntermediateEntries() { return hashShuffleIntermediateEntries; } + protected void waitingIntermediateReport() { + LOG.info(getId() + ", waiting IntermediateReport: expectedTaskNum=" + completeReportReceived.get()); + synchronized(completeReportReceived) { + startTime = System.currentTimeMillis(); + while (true) { + if (completeReportReceived.get() >= tasks.size()) { + LOG.info(getId() + ", completed waiting IntermediateReport"); + return; + } else { + try { + completeReportReceived.wait(10 * 1000); + } catch (InterruptedException e) { + } + long elapsedTime = System.currentTimeMillis() - startTime; + if (elapsedTime >= 120 * 1000) { + LOG.error(getId() + ", Timeout while receiving intermediate reports: " + elapsedTime + " ms"); + abort(SubQueryState.FAILED); + return; + } + } + } + } + } + + public void receiveExecutionBlockReport(TajoWorkerProtocol.ExecutionBlockReport report) { + LOG.info(getId() + ", receiveExecutionBlockReport:" + report.getSucceededTasks()); + if (!report.getReportSuccess()) { + LOG.error(getId() + ", ExecutionBlock final report fail cause:" + report.getReportErrorMessage()); + abort(SubQueryState.FAILED); + return; + } + if (report.getIntermediateEntriesCount() > 0) { + synchronized (hashShuffleIntermediateEntries) { + for (IntermediateEntryProto eachInterm: report.getIntermediateEntriesList()) { + if (getId().getId() == 7) { + LOG.fatal(">>>>>receiveExecutionBlockReport>" + eachInterm.getPartId() + "," + eachInterm.getVolume()); + for(IntermediateEntryProto.PageProto eachPage: eachInterm.getPagesList()) { + LOG.fatal(">>>receiveExecutionBlockReport>Page:" + eachPage.getPos() + "," + eachPage.getLength()); + } + } + hashShuffleIntermediateEntries.add(new IntermediateEntry(eachInterm)); + } + } + } + synchronized(completeReportReceived) { + completeReportReceived.addAndGet(report.getSucceededTasks()); + completeReportReceived.notifyAll(); + } + } + private static class SubQueryCompleteTransition implements MultipleArcTransition { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index 126340ef24..37b3541553 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -35,6 +35,7 @@ import org.apache.tajo.catalog.CatalogClient; import org.apache.tajo.catalog.CatalogService; import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.ipc.QueryMasterProtocol; import org.apache.tajo.ipc.TajoMasterProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol.ExecutionBlockReport; import org.apache.tajo.ipc.TajoWorkerProtocol.FailureIntermediateProto; @@ -349,7 +350,7 @@ public void stop() { public class WorkerContext { public QueryMaster getQueryMaster() { - if(queryMasterManagerService == null) { + if (queryMasterManagerService == null) { return null; } return queryMasterManagerService.getQueryMaster(); @@ -384,28 +385,29 @@ public int getHttpPort() { } public String getWorkerName() { - if(queryMasterMode) { + if (queryMasterMode) { return getQueryMasterManagerService().getHostAndPort(); } else { return getTajoWorkerManagerService().getHostAndPort(); } } + public void stopWorker(boolean force) { stop(); - if(force) { + if (force) { System.exit(0); } } protected void cleanup(String strPath) { - if(deletionService == null) return; + if (deletionService == null) return; LocalDirAllocator lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); try { Iterable iter = lDirAllocator.getAllLocalPathsToRead(strPath, systemConf); FileSystem localFS = FileSystem.getLocal(systemConf); - for (Path path : iter){ + for (Path path : iter) { deletionService.delete(localFS.makeQualified(path)); } } catch (IOException e) { @@ -414,21 +416,21 @@ protected void cleanup(String strPath) { } protected void cleanupTemporalDirectories() { - if(deletionService == null) return; + if (deletionService == null) return; LocalDirAllocator lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); try { Iterable iter = lDirAllocator.getAllLocalPathsToRead(".", systemConf); FileSystem localFS = FileSystem.getLocal(systemConf); - for (Path path : iter){ + for (Path path : iter) { PathData[] items = PathData.expandAsGlob(localFS.makeQualified(new Path(path, "*")).toString(), systemConf); ArrayList paths = new ArrayList(); - for (PathData pd : items){ + for (PathData pd : items) { paths.add(pd.path); } - if(paths.size() == 0) continue; + if (paths.size() == 0) continue; deletionService.delete(null, paths.toArray(new Path[paths.size()])); } @@ -450,13 +452,13 @@ public int getNumClusterNodes() { } public void setClusterResource(TajoMasterProtocol.ClusterResourceSummary clusterResource) { - synchronized(numClusterNodes) { + synchronized (numClusterNodes) { TajoWorker.this.clusterResource = clusterResource; } } public TajoMasterProtocol.ClusterResourceSummary getClusterResource() { - synchronized(numClusterNodes) { + synchronized (numClusterNodes) { return TajoWorker.this.clusterResource; } } @@ -488,63 +490,6 @@ public TajoSystemMetrics getWorkerSystemMetrics() { public HashShuffleAppenderManager getHashShuffleAppenderManager() { return hashShuffleAppenderManager; } - - public ExecutionBlockReport closeHashShuffle(ExecutionBlockId ebId) { - ExecutionBlockReport.Builder reporterBuilder = ExecutionBlockReport.newBuilder(); - reporterBuilder.setEbId(ebId.getProto()); - reporterBuilder.setReportSuccess(true); - try { - List intermediateEntries = new ArrayList(); - List shuffles = hashShuffleAppenderManager.close(ebId); - if (shuffles == null) { - reporterBuilder.addAllIntermediateEntries(intermediateEntries); - return reporterBuilder.build(); - } - - IntermediateEntryProto.Builder intermediateBuilder = IntermediateEntryProto.newBuilder(); - PageProto.Builder pageBuilder = PageProto.newBuilder(); - FailureIntermediateProto.Builder failureBuilder = FailureIntermediateProto.newBuilder(); - - for (HashShuffleIntermediate eachShuffle: shuffles) { - List pages = new ArrayList(); - List failureIntermediateItems = new ArrayList(); - - for (Pair eachPage: eachShuffle.getPages()) { - pageBuilder.clear(); - pageBuilder.setPos(eachPage.getFirst()); - pageBuilder.setLength(eachPage.getSecond()); - pages.add(pageBuilder.build()); - } - - for(Pair> eachFailure: eachShuffle.getFailureTskTupleIndexes()) { - failureBuilder.clear(); - failureBuilder.setPagePos(eachFailure.getFirst()); - failureBuilder.setStartRowNum(eachFailure.getSecond().getFirst()); - failureBuilder.setEndRowNum(eachFailure.getSecond().getSecond()); - failureIntermediateItems.add(failureBuilder.build()); - } - intermediateBuilder.clear(); - intermediateBuilder.setEbId(ebId.getProto()) - .setHost(workerContext.getTajoWorkerManagerService().getBindAddr().getHostName() + ":" + getPullService().getPort()) - .setTaskId(-1) - .setAttemptId(-1) - .setPartId(eachShuffle.getPartId()) - .setVolume(eachShuffle.getVolume()) - .addAllPages(pages) - .addAllFailures(failureIntermediateItems); - intermediateEntries.add(intermediateBuilder.build()); - } - - // send intermediateEntries to QueryMaster - reporterBuilder.addAllIntermediateEntries(intermediateEntries); - - } catch (Exception e) { - LOG.error(e.getMessage(), e); - reporterBuilder.setReportSuccess(false); - reporterBuilder.setReportErrorMessage(e.getMessage()); - } - return reporterBuilder.build(); - } } public void stopWorkerForce() { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java index 04f0165c8d..c5f14468cf 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java @@ -157,18 +157,13 @@ public void cleanup(RpcController controller, TajoIdProtos.QueryIdProto request, @Override public void cleanupExecutionBlocks(RpcController controller, TajoWorkerProtocol.ExecutionBlockListProto ebIds, - RpcCallback done) { - ExecutionBlockReport report = null; + RpcCallback done) { for (TajoIdProtos.ExecutionBlockIdProto executionBlockIdProto : ebIds.getExecutionBlockIdList()) { - if (report == null) { - report = workerContext.closeHashShuffle(new ExecutionBlockId(executionBlockIdProto)); - continue; - } String inputDir = TaskRunner.getBaseInputDir(new ExecutionBlockId(executionBlockIdProto)).toString(); workerContext.cleanup(inputDir); String outputDir = TaskRunner.getBaseOutputDir(new ExecutionBlockId(executionBlockIdProto)).toString(); workerContext.cleanup(outputDir); } - done.run(report); + done.run(TajoWorker.TRUE_PROTO); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java index 8f7322377e..32cd4f5ed2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java @@ -41,6 +41,7 @@ import org.apache.tajo.engine.utils.TupleCache; import org.apache.tajo.ipc.QueryMasterProtocol; import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService; +import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.rpc.CallFuture; import org.apache.tajo.rpc.NettyClientBase; import org.apache.tajo.rpc.NullCallback; @@ -166,6 +167,18 @@ public TaskRunner(TaskRunnerManager taskRunnerManager, TajoConf conf, String[] a } } + protected void sendExecutionBlockReport(TajoWorkerProtocol.ExecutionBlockReport reporter) throws Exception { + QueryMasterProtocol.QueryMasterProtocolService.Interface qmClientService = null; + NettyClientBase qmClient = null; + try { + qmClient = connPool.getConnection(qmMasterAddr, QueryMasterProtocol.class, true); + qmClientService = qmClient.getStub(); + qmClientService.doneExecutionBlock(null, reporter, NullCallback.get()); + } finally { + connPool.releaseConnection(qmClient); + } + } + public String getId() { return getId(executionBlockId, containerId); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java index 24470efc5b..f08ca361bf 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java @@ -23,11 +23,17 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.CompositeService; +import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.QueryUnitAttemptId; import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.ipc.QueryMasterProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.storage.HashShuffleAppenderManager; +import org.apache.tajo.util.Pair; import java.util.*; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; public class TaskRunnerManager extends CompositeService { private static final Log LOG = LogFactory.getLog(TaskRunnerManager.class); @@ -90,17 +96,31 @@ public void stopTask(String id) { synchronized(taskRunnerMap) { TaskRunner taskRunner = taskRunnerMap.remove(id); if (taskRunner != null) { -// synchronized(taskRunnerCompleteCounter) { -// ExecutionBlockId ebId = taskRunner.getContext().getExecutionBlockId(); -// Pair counter = taskRunnerCompleteCounter.get(ebId); -// if (counter != null) { -// if (counter.getSecond().decrementAndGet() <= 0) { -// LOG.info(ebId + "'s all tasks are completed."); -// closeHashShuffle(ebId, counter.getFirst().get()); -// taskRunnerCompleteCounter.remove(ebId); -// } -// } -// } + synchronized(taskRunnerCompleteCounter) { + ExecutionBlockId ebId = taskRunner.getContext().getExecutionBlockId(); + AtomicInteger ebSuccessedTaskNums = successedTaskNums.get(ebId); + if (ebSuccessedTaskNums == null) { + ebSuccessedTaskNums = new AtomicInteger(taskRunner.getContext().succeededTasksNum.get()); + successedTaskNums.put(ebId, ebSuccessedTaskNums); + } else { + ebSuccessedTaskNums.addAndGet(taskRunner.getContext().succeededTasksNum.get()); + } + + Pair counter = taskRunnerCompleteCounter.get(ebId); + + if (counter != null) { + if (counter.getSecond().decrementAndGet() <= 0) { + LOG.info(ebId + "'s all tasks are completed."); + try { + closeExecutionBlock(ebId, ebSuccessedTaskNums.get(), taskRunner); + } catch (Exception e) { + LOG.error(ebId + ", closing error:" + e.getMessage(), e); + } + successedTaskNums.remove(ebId); + taskRunnerCompleteCounter.remove(ebId); + } + } + } } } if(workerContext.isYarnContainerMode()) { @@ -108,6 +128,80 @@ public void stopTask(String id) { } } + private void closeExecutionBlock(ExecutionBlockId ebId, int succeededTasks, TaskRunner lastTaskRunner) throws Exception { + TajoWorkerProtocol.ExecutionBlockReport.Builder reporterBuilder = + TajoWorkerProtocol.ExecutionBlockReport.newBuilder(); + reporterBuilder.setEbId(ebId.getProto()); + reporterBuilder.setReportSuccess(true); + reporterBuilder.setSucceededTasks(succeededTasks); + try { + List intermediateEntries = + new ArrayList(); + List shuffles = + workerContext.getHashShuffleAppenderManager().close(ebId); + if (shuffles == null) { + reporterBuilder.addAllIntermediateEntries(intermediateEntries); + lastTaskRunner.sendExecutionBlockReport(reporterBuilder.build()); + return; + } + + TajoWorkerProtocol.IntermediateEntryProto.Builder intermediateBuilder = + TajoWorkerProtocol.IntermediateEntryProto.newBuilder(); + TajoWorkerProtocol.IntermediateEntryProto.PageProto.Builder pageBuilder = + TajoWorkerProtocol.IntermediateEntryProto.PageProto.newBuilder(); + TajoWorkerProtocol.FailureIntermediateProto.Builder failureBuilder = + TajoWorkerProtocol.FailureIntermediateProto.newBuilder(); + + for (HashShuffleAppenderManager.HashShuffleIntermediate eachShuffle: shuffles) { + if (ebId.getId() == 7) { + LOG.fatal(">>>>>" + eachShuffle.getPartId() + "," + eachShuffle.getVolume()); + for(Pair eachPage: eachShuffle.getPages()) { + LOG.fatal(">>>Page:" + eachPage); + } + } + List pages = + new ArrayList(); + List failureIntermediateItems = + new ArrayList(); + + for (Pair eachPage: eachShuffle.getPages()) { + pageBuilder.clear(); + pageBuilder.setPos(eachPage.getFirst()); + pageBuilder.setLength(eachPage.getSecond()); + pages.add(pageBuilder.build()); + } + + for(Pair> eachFailure: eachShuffle.getFailureTskTupleIndexes()) { + failureBuilder.clear(); + failureBuilder.setPagePos(eachFailure.getFirst()); + failureBuilder.setStartRowNum(eachFailure.getSecond().getFirst()); + failureBuilder.setEndRowNum(eachFailure.getSecond().getSecond()); + failureIntermediateItems.add(failureBuilder.build()); + } + intermediateBuilder.clear(); + intermediateBuilder.setEbId(ebId.getProto()) + .setHost(workerContext.getTajoWorkerManagerService().getBindAddr().getHostName() + ":" + + workerContext.getPullService().getPort()) + .setTaskId(-1) + .setAttemptId(-1) + .setPartId(eachShuffle.getPartId()) + .setVolume(eachShuffle.getVolume()) + .addAllPages(pages) + .addAllFailures(failureIntermediateItems); + intermediateEntries.add(intermediateBuilder.build()); + } + + // send intermediateEntries to QueryMaster + reporterBuilder.addAllIntermediateEntries(intermediateEntries); + + } catch (Exception e) { + LOG.error(e.getMessage(), e); + reporterBuilder.setReportSuccess(false); + reporterBuilder.setReportErrorMessage(e.getMessage()); + } + lastTaskRunner.sendExecutionBlockReport(reporterBuilder.build()); + } + public Collection getTaskRunners() { synchronized(taskRunnerMap) { return Collections.unmodifiableCollection(taskRunnerMap.values()); @@ -159,9 +253,11 @@ public int getNumTasks() { } } - // <#t asks, # running tasks> -// Map> taskRunnerCompleteCounter = -// new HashMap>(); + //<# tasks, # running tasks> + Map> taskRunnerCompleteCounter = + new HashMap>(); + + Map successedTaskNums = new HashMap(); public void startTask(final String[] params) { //TODO change to use event dispatcher @@ -179,16 +275,16 @@ public void run() { taskRunnerHistoryMap.put(taskRunner.getId(), taskRunner.getContext().getExcutionBlockHistory()); } -// synchronized(taskRunnerCompleteCounter) { -// ExecutionBlockId ebId = taskRunner.getContext().getExecutionBlockId(); -// Pair counter = taskRunnerCompleteCounter.get(ebId); -// if (counter == null) { -// counter = new Pair(new AtomicInteger(0), new AtomicInteger(0)); -// taskRunnerCompleteCounter.put(ebId, counter); -// } -// counter.getFirst().incrementAndGet(); -// counter.getSecond().incrementAndGet(); -// } + synchronized(taskRunnerCompleteCounter) { + ExecutionBlockId ebId = taskRunner.getContext().getExecutionBlockId(); + Pair counter = taskRunnerCompleteCounter.get(ebId); + if (counter == null) { + counter = new Pair(new AtomicInteger(0), new AtomicInteger(0)); + taskRunnerCompleteCounter.put(ebId, counter); + } + counter.getFirst().incrementAndGet(); + counter.getSecond().incrementAndGet(); + } taskRunner.init(systemConf); taskRunner.start(); } catch (Exception e) { diff --git a/tajo-core/src/main/proto/QueryMasterProtocol.proto b/tajo-core/src/main/proto/QueryMasterProtocol.proto index ebc54a4f9c..0119a882c3 100644 --- a/tajo-core/src/main/proto/QueryMasterProtocol.proto +++ b/tajo-core/src/main/proto/QueryMasterProtocol.proto @@ -35,7 +35,7 @@ service QueryMasterProtocolService { rpc ping (QueryUnitAttemptIdProto) returns (BoolProto); rpc fatalError(TaskFatalErrorReport) returns (BoolProto); rpc done (TaskCompletionReport) returns (BoolProto); - //rpc doneExecutionBlock(ExecutionBlockCompeletReporter) returns (BoolProto); + rpc doneExecutionBlock(ExecutionBlockReport) returns (BoolProto); //from TajoMaster's QueryJobManager rpc killQuery(QueryIdProto) returns (BoolProto); diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto index b6638aacb8..cdb1438a23 100644 --- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto +++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto @@ -118,7 +118,8 @@ message ExecutionBlockReport { required ExecutionBlockIdProto ebId = 1; required bool reportSuccess = 2; optional string reportErrorMessage = 3; - repeated IntermediateEntryProto intermediateEntries = 4; + required int32 succeededTasks = 4; + repeated IntermediateEntryProto intermediateEntries = 5; } message QueryUnitResponseProto { @@ -219,7 +220,7 @@ service TajoWorkerProtocolService { rpc executeExecutionBlock(RunExecutionBlockRequestProto) returns (BoolProto); rpc killTaskAttempt(QueryUnitAttemptIdProto) returns (BoolProto); rpc cleanup(QueryIdProto) returns (BoolProto); - rpc cleanupExecutionBlocks(ExecutionBlockListProto) returns (ExecutionBlockReport); + rpc cleanupExecutionBlocks(ExecutionBlockListProto) returns (BoolProto); } message EnforceProperty { diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java index ceb9684ab5..c340dcf4e8 100644 --- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java +++ b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java @@ -466,7 +466,8 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) sendError(ctx, errorMessage, BAD_REQUEST); return; } - LOG.info("RequestURL" + request.getUri() + ", fileLen=" + file.length()); + LOG.info("RequestURL: " + request.getUri() + ", fileLen=" + file.length()); + LOG.fatal(">>>>>>>>>>>>>>>>startPos:" + startPos + ", " + length); FileChunk chunk = new FileChunk(file, startPos, readLen); chunks.add(chunk); } else { From 2d49339111be67d058158431a94680ab2749000d Mon Sep 17 00:00:00 2001 From: HyoungJun Kim Date: Tue, 12 Aug 2014 15:09:51 +0900 Subject: [PATCH 5/8] TAJO-992: Reduce number of hash shuffle output file. Remove unused log --- .../org/apache/tajo/master/querymaster/SubQuery.java | 10 +--------- .../org/apache/tajo/worker/TaskRunnerManager.java | 6 ------ .../org/apache/tajo/storage/HashShuffleAppender.java | 12 +++++++----- .../tajo/storage/HashShuffleAppenderManager.java | 2 +- .../tajo/pullserver/TajoPullServerService.java | 1 - 5 files changed, 9 insertions(+), 22 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java index e7732e7b9d..96027e1013 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java @@ -298,7 +298,6 @@ public AbstractTaskScheduler getTaskScheduler() { } public void setStartTime() { - LOG.fatal("setStartTime>>>>>>>>>>>>>" + startTime + ">" + context.getClock().getTime()); startTime = context.getClock().getTime(); } @@ -308,7 +307,6 @@ public long getStartTime() { } public void setFinishTime() { - LOG.fatal("setFinishTime>>>>>>>>>>>>>" + startTime + ">" + context.getClock().getTime()); finishTime = context.getClock().getTime(); } @@ -1127,7 +1125,7 @@ public List getHashShuffleIntermediateEntries() { protected void waitingIntermediateReport() { LOG.info(getId() + ", waiting IntermediateReport: expectedTaskNum=" + completeReportReceived.get()); synchronized(completeReportReceived) { - startTime = System.currentTimeMillis(); + long startTime = System.currentTimeMillis(); while (true) { if (completeReportReceived.get() >= tasks.size()) { LOG.info(getId() + ", completed waiting IntermediateReport"); @@ -1158,12 +1156,6 @@ public void receiveExecutionBlockReport(TajoWorkerProtocol.ExecutionBlockReport if (report.getIntermediateEntriesCount() > 0) { synchronized (hashShuffleIntermediateEntries) { for (IntermediateEntryProto eachInterm: report.getIntermediateEntriesList()) { - if (getId().getId() == 7) { - LOG.fatal(">>>>>receiveExecutionBlockReport>" + eachInterm.getPartId() + "," + eachInterm.getVolume()); - for(IntermediateEntryProto.PageProto eachPage: eachInterm.getPagesList()) { - LOG.fatal(">>>receiveExecutionBlockReport>Page:" + eachPage.getPos() + "," + eachPage.getLength()); - } - } hashShuffleIntermediateEntries.add(new IntermediateEntry(eachInterm)); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java index f08ca361bf..ec413b2e27 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java @@ -153,12 +153,6 @@ private void closeExecutionBlock(ExecutionBlockId ebId, int succeededTasks, Task TajoWorkerProtocol.FailureIntermediateProto.newBuilder(); for (HashShuffleAppenderManager.HashShuffleIntermediate eachShuffle: shuffles) { - if (ebId.getId() == 7) { - LOG.fatal(">>>>>" + eachShuffle.getPartId() + "," + eachShuffle.getVolume()); - for(Pair eachPage: eachShuffle.getPages()) { - LOG.fatal(">>>Page:" + eachPage); - } - } List pages = new ArrayList(); List failureIntermediateItems = diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java index c4b89c1b72..934fd941dd 100644 --- a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java +++ b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java @@ -156,11 +156,13 @@ public void close() throws IOException { nextPage(offset); } appender.close(); - if (!pages.isEmpty()) { - LOG.info(ebId + ",partId=" + partId + " Appender closed: fileLen=" + offset + ", pages=" + pages.size() - + ", lastPage=" + pages.get(pages.size() - 1)); - } else { - LOG.info(ebId + ",partId=" + partId + " Appender closed: fileLen=" + offset + ", pages=" + pages.size()); + if (LOG.isDebugEnabled()) { + if (!pages.isEmpty()) { + LOG.info(ebId + ",partId=" + partId + " Appender closed: fileLen=" + offset + ", pages=" + pages.size() + + ", lastPage=" + pages.get(pages.size() - 1)); + } else { + LOG.info(ebId + ",partId=" + partId + " Appender closed: fileLen=" + offset + ", pages=" + pages.size()); + } } closed.set(true); tableStats = appender.getStats(); diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java index e9555dd83d..228c646303 100644 --- a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java +++ b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java @@ -112,7 +112,7 @@ private Path getDataFile(ExecutionBlockId ebId, int partId) throws IOException { // the base dir for an output dir String executionBlockBaseDir = ebId.getQueryId().toString() + "/output" + "/" + ebId.getId() + "/hash-shuffle"; Path baseDirPath = localFS.makeQualified(lDirAllocator.getLocalPathForWrite(executionBlockBaseDir, systemConf)); - LOG.info(ebId + "'s basedir is created (" + baseDirPath + ")"); + //LOG.info(ebId + "'s basedir is created (" + baseDirPath + ")"); // If EB has many partition, too many shuffle file are in single directory. return StorageUtil.concatPath(baseDirPath, "" + getPartParentId(partId, systemConf), "" + partId); diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java index c340dcf4e8..02fbc19e36 100644 --- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java +++ b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java @@ -467,7 +467,6 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) return; } LOG.info("RequestURL: " + request.getUri() + ", fileLen=" + file.length()); - LOG.fatal(">>>>>>>>>>>>>>>>startPos:" + startPos + ", " + length); FileChunk chunk = new FileChunk(file, startPos, readLen); chunks.add(chunk); } else { From 98e6314ab4974453035647f2bc78940fcb096d9e Mon Sep 17 00:00:00 2001 From: HyoungJun Kim Date: Wed, 13 Aug 2014 21:36:06 +0900 Subject: [PATCH 6/8] TAJO-992: Reduce number of hash shuffle output file. Fix a wrong calculation of Bytes in StorageUnit --- .../java/org/apache/tajo/unit/StorageUnit.java | 2 +- .../tajo/master/querymaster/Repartitioner.java | 14 +++++++++----- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/tajo-common/src/main/java/org/apache/tajo/unit/StorageUnit.java b/tajo-common/src/main/java/org/apache/tajo/unit/StorageUnit.java index 8cde6592f7..dd345c18c5 100644 --- a/tajo-common/src/main/java/org/apache/tajo/unit/StorageUnit.java +++ b/tajo-common/src/main/java/org/apache/tajo/unit/StorageUnit.java @@ -21,7 +21,7 @@ public class StorageUnit { public static final int B = 8; - public static final int KB = B * 1024; + public static final int KB = 1024; public static final int MB = KB * 1024; public static final int GB = MB * 1024; public static final int TB = GB * 1024; diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java index 7b17910606..da26f3455a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java @@ -919,11 +919,13 @@ public static void scheduleScatteredHashShuffleFetches(TaskSchedulerContext sche "tajo.shuffle.hash.appender.page.volumn-mb"); } List> fetches = new ArrayList>(); - + + long totalIntermediateSize = 0L; for (Entry> listEntry : intermediates.entrySet()) { // merge by PartitionId Map> partitionIntermMap = new HashMap>(); for (IntermediateEntry eachInterm: listEntry.getValue()) { + totalIntermediateSize += eachInterm.getVolume(); int partId = eachInterm.getPartId(); List partitionInterms = partitionIntermMap.get(partId); if (partitionInterms == null) { @@ -934,9 +936,9 @@ public static void scheduleScatteredHashShuffleFetches(TaskSchedulerContext sche } } - // Grouping or splitting to fit DIST_QUERY_TABLE_PARTITION_VOLUME size - for (List partitionEntires : partitionIntermMap.values()) { - List> eachFetches = splitOrMergeIntermediates(listEntry.getKey(), partitionEntires, + // Grouping or splitting to fit $DIST_QUERY_TABLE_PARTITION_VOLUME size + for (List partitionEntries : partitionIntermMap.values()) { + List> eachFetches = splitOrMergeIntermediates(listEntry.getKey(), partitionEntries, splitVolume, pageSize); if (eachFetches != null && !eachFetches.isEmpty()) { fetches.addAll(eachFetches); @@ -958,7 +960,9 @@ public static void scheduleScatteredHashShuffleFetches(TaskSchedulerContext sche LOG.info(subQuery.getId() + ", ShuffleType:" + SCATTERED_HASH_SHUFFLE.name() - + ", DeterminedTaskNum : " + fetches.size()); + + ", Intermediate Size: " + totalIntermediateSize + + ", splitSize: " + splitVolume + + ", DeterminedTaskNum: " + fetches.size()); } /** From 2a4901ca592c9e41f9329a7b085a7ae539cceafe Mon Sep 17 00:00:00 2001 From: HyoungJun Kim Date: Mon, 18 Aug 2014 17:56:26 +0900 Subject: [PATCH 7/8] TAJO-992: Reduce number of hash shuffle output file. Remove unused code in HashShuffleAppenderManager. --- .../main/java/org/apache/tajo/worker/Task.java | 2 +- .../storage/HashShuffleAppenderManager.java | 17 +---------------- .../tajo/pullserver/TajoPullServerService.java | 1 - 3 files changed, 2 insertions(+), 18 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index cff3ecbfb0..1881685dfa 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -436,7 +436,7 @@ public void run() { } finally { context.setProgress(1.0f); taskRunnerContext.completedTasksNum.incrementAndGet(); - context.getHashShuffleAppenderManager().taskFinished(taskId); + context.getHashShuffleAppenderManager().finalizeTask(taskId); if (killed || aborted) { context.setExecutorProgress(0.0f); diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java index 228c646303..f0699b7b45 100644 --- a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java +++ b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java @@ -122,21 +122,6 @@ private Path getDataFile(ExecutionBlockId ebId, int partId) throws IOException { } } - public Path getPartitionAppenderDataFile(ExecutionBlockId ebId, int partId) { - synchronized (appenderMap) { - Map partitionAppenderMap = appenderMap.get(ebId); - if (partitionAppenderMap != null) { - PartitionAppenderMeta meta = partitionAppenderMap.get(partId); - if (meta != null) { - return meta.dataFile; - } - } - } - - LOG.warn("Can't find HashShuffleAppender:" + ebId + ", part=" + partId); - return null; - } - public List close(ExecutionBlockId ebId) throws IOException { Map partitionAppenderMap = null; synchronized (appenderMap) { @@ -169,7 +154,7 @@ public List close(ExecutionBlockId ebId) throws IOExcep return intermEntries; } - public void taskFinished(QueryUnitAttemptId taskId) { + public void finalizeTask(QueryUnitAttemptId taskId) { synchronized (appenderMap) { Map partitionAppenderMap = appenderMap.get(taskId.getQueryUnitId().getExecutionBlockId()); diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java index 02fbc19e36..e68e35148e 100644 --- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java +++ b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java @@ -208,7 +208,6 @@ public void init(Configuration conf) { selector = RpcChannelFactory.createServerChannelFactory("PullServerAuxService", workerNum); localFS = new LocalFileSystem(); - //super.init(new Configuration(conf)); super.init(conf); this.getConfig().setInt(TajoConf.ConfVars.PULLSERVER_PORT.varname From 9f70a72a591b6a7b034474363e1144ddb7423265 Mon Sep 17 00:00:00 2001 From: HyoungJun Kim Date: Wed, 20 Aug 2014 18:20:49 +0900 Subject: [PATCH 8/8] TAJO-992: Reduce number of hash shuffle output file. Add some comments about Intermediate file's split. --- .../java/org/apache/tajo/master/querymaster/Repartitioner.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java index da26f3455a..4deddee587 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java @@ -991,6 +991,9 @@ public static List> splitOrMergeIntermediates( if (firstSplitVolume < pageSize) { firstSplitVolume = splitVolume; } + + //Each Pair object in the splits variable is assigned to the next ExectionBlock's task. + //The first long value is a offset of the intermediate file and the second long value is length. List> splits = currentInterm.split(firstSplitVolume, splitVolume); if (splits == null || splits.isEmpty()) { break;