You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-issues@hadoop.apache.org by "BELUGA BEHR (JIRA)" <ji...@apache.org> on 2018/02/20 17:56:02 UTC

[jira] [Commented] (MAPREDUCE-7057) MergeThread Review

    [ https://issues.apache.org/jira/browse/MAPREDUCE-7057?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16370359#comment-16370359 ] 

BELUGA BEHR commented on MAPREDUCE-7057:
----------------------------------------

Added a new implementation which uses the java.concurrent package.  It blocks new inputs to the queue if the queue is closed and only removes items from the 'input' set if its guaranteed to be submitted into the queue.  It also gives warning errors about anything that may be erroneously left in the queue.

{code}
## My Implementation

Result "org.apache.hadoop.mapreduce.task.reduce.MergeThreadSpeedTest.testSpeed":
  51669734.114 ±(99.9%) 178997.890 ops/s [Average]
  (min, avg, max) = (47854965.698, 51669734.114, 52568098.484), stdev = 757888.276
  CI (99.9%): [51490736.224, 51848732.004] (assumes normal distribution)


# Run complete. Total time: 00:06:45

Benchmark                        Mode  Cnt         Score        Error  Units
MergeThreadSpeedTest.testSpeed  thrpt  200  51669734.114 ± 178997.890  ops/s

## Hadoop Implementation

Result "org.apache.hadoop.mapreduce.task.reduce.MergeThreadSpeedTest.testSpeed":
  14711295.937 ±(99.9%) 68088.455 ops/s [Average]
  (min, avg, max) = (12859333.389, 14711295.937, 15047678.326), stdev = 288290.781
  CI (99.9%): [14643207.482, 14779384.392] (assumes normal distribution)


# Run complete. Total time: 00:06:45

Benchmark                        Mode  Cnt         Score       Error  Units
MergeThreadSpeedTest.testSpeed  thrpt  200  14711295.937 ± 68088.455  ops/s
{code}

> MergeThread Review
> ------------------
>
>                 Key: MAPREDUCE-7057
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-7057
>             Project: Hadoop Map/Reduce
>          Issue Type: Improvement
>          Components: mrv2
>    Affects Versions: 3.0.0
>            Reporter: BELUGA BEHR
>            Priority: Minor
>         Attachments: MAPREDUCE-7057.1.patch
>
>
> Source:
>  [MergeThread.java|https://github.com/apache/hadoop/blob/178751ed8c9d47038acf8616c226f1f52e884feb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeThread.java]
> Update this class to use Java 1.8 concurrent package.  There also some corner-cases not being addressed with the current implementation:
> {code:java|title=MergeThread.java}
> // There is a scenario here where N threads have submitted inputs and are all waiting for the 'pendingToBeMerged' object.  At this point, imagine the 'close' method is called.  The close method will run, see nothing in the queue, interrupt the processing thread, and cause it to exit.  Afterwards, the 'startMerge' threads will all be triggered and add the inputs to a queue for which there is no consumer.  At this point, the T items have been removed from the inputs with no way to recover them.  In practice, this may not ever be the case, but it can be tightened up.
>   public void startMerge(Set<T> inputs) {
>     if (!closed) {
>       numPending.incrementAndGet();
>       List<T> toMergeInputs = new ArrayList<T>();
>       Iterator<T> iter=inputs.iterator();
>       for (int ctr = 0; iter.hasNext() && ctr < mergeFactor; ++ctr) {
>         toMergeInputs.add(iter.next());
>         iter.remove();
>       }
>       LOG.info(getName() + ": Starting merge with " + toMergeInputs.size() + 
>                " segments, while ignoring " + inputs.size() + " segments");
>       synchronized(pendingToBeMerged) {
>         pendingToBeMerged.addLast(toMergeInputs);
>         pendingToBeMerged.notifyAll();
>       }
>     }
>   }
>   public synchronized void close() throws InterruptedException {
>     closed = true;
>     waitForMerge();
>     interrupt();
>   }
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: mapreduce-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: mapreduce-issues-help@hadoop.apache.org