Uploaded image for project: 'Tajo'
  1. Tajo
  2. TAJO-1950

Query master uses too much memory during range shuffle

    Details

      Description

      I ran a simple sort query on a 8TB table as follows.

      tpch10tb> select * from lineitem order by l_orderkey;
      

      After the first stage is completed, query master divides the range of the sort key (l_orderkey) into multiple partitions for range shuffle. Here, the partitioning time took about 9 minutes.

      Here is the log.

      ...
      2015-10-26 14:23:10,782 INFO org.apache.tajo.engine.planner.global.ParallelExecutionQueue: Next executable block eb_1445835438802_0004_000002
      2015-10-26 14:23:10,782 INFO org.apache.tajo.querymaster.Query: Scheduling Stage:eb_1445835438802_0004_000002
      2015-10-26 14:23:10,796 INFO org.apache.tajo.querymaster.Stage: org.apache.tajo.querymaster.DefaultTaskScheduler is chosen for the task scheduling for eb_1445835438802_0004_000002
      2015-10-26 14:23:10,796 INFO org.apache.tajo.querymaster.Stage: eb_1445835438802_0004_000002, Table's volume is approximately 663647 MB
      2015-10-26 14:23:10,796 INFO org.apache.tajo.querymaster.Stage: eb_1445835438802_0004_000002, The determined number of non-leaf tasks is 10370
      2015-10-26 14:23:10,816 INFO org.apache.tajo.querymaster.Repartitioner: eb_1445835438802_0004_000002, Try to divide [(6000000000), (1)) into 10370 sub ranges (total units: 10370)
      2015-10-26 14:24:58,996 INFO org.apache.tajo.util.JvmPauseMonitor: Detected pause in JVM or host machine (eg GC): pause of approximately 2440ms
      GC pool 'PS MarkSweep' had collection(s): count=1 time=2214ms
      GC pool 'PS Scavenge' had collection(s): count=1 time=622ms
      2015-10-26 14:27:24,040 WARN org.apache.tajo.util.JvmPauseMonitor: Detected pause in JVM or host machine (eg GC): pause of approximately 13237ms
      GC pool 'PS MarkSweep' had collection(s): count=1 time=12635ms
      GC pool 'PS Scavenge' had collection(s): count=1 time=674ms
      2015-10-26 14:28:51,914 WARN org.apache.tajo.util.JvmPauseMonitor: Detected pause in JVM or host machine (eg GC): pause of approximately 20873ms
      GC pool 'PS MarkSweep' had collection(s): count=1 time=20486ms
      GC pool 'PS Scavenge' had collection(s): count=1 time=644ms
      2015-10-26 14:30:52,392 WARN org.apache.tajo.util.JvmPauseMonitor: Detected pause in JVM or host machine (eg GC): pause of approximately 30986ms
      GC pool 'PS MarkSweep' had collection(s): count=1 time=30546ms
      GC pool 'PS Scavenge' had collection(s): count=1 time=696ms
      2015-10-26 14:32:07,550 WARN org.apache.tajo.util.JvmPauseMonitor: Detected pause in JVM or host machine (eg GC): pause of approximately 15449ms
      GC pool 'PS MarkSweep' had collection(s): count=1 time=14593ms
      GC pool 'PS Scavenge' had collection(s): count=1 time=1148ms
      2015-10-26 14:32:15,807 INFO org.apache.tajo.querymaster.Stage: 10370 objects are scheduled
      ...
      

        Issue Links

          Activity

          Hide
          hudson Hudson added a comment -

          SUCCESS: Integrated in Tajo-0.11.1-build #141 (See https://builds.apache.org/job/Tajo-0.11.1-build/141/)
          TAJO-1950: Query master uses too much memory during range shuffle. (fix (jihoonson: rev b9f2d4ef9b38cd0a4c0b7a04667e2b1e23bf873b)

          • tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
          • tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java
          Show
          hudson Hudson added a comment - SUCCESS: Integrated in Tajo-0.11.1-build #141 (See https://builds.apache.org/job/Tajo-0.11.1-build/141/ ) TAJO-1950 : Query master uses too much memory during range shuffle. (fix (jihoonson: rev b9f2d4ef9b38cd0a4c0b7a04667e2b1e23bf873b) tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java
          Hide
          hudson Hudson added a comment -

          SUCCESS: Integrated in Tajo-master-build #1038 (See https://builds.apache.org/job/Tajo-master-build/1038/)
          TAJO-1950: Query master uses too much memory during range shuffle. (jihoonson: rev 1f9ae1da0424731567cea18e975c47d4479b0ae9)

          • tajo-core-tests/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java
          • tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
          • tajo-core-tests/src/test/java/org/apache/tajo/master/TestRepartitioner.java
          • tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
          • tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcher.java
          • tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java
          • tajo-core/src/main/proto/ResourceProtos.proto
          • tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
          • tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
          • tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
          • tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java
          • CHANGES
          • tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
          • tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
          • tajo-core/src/main/resources/webapps/worker/task.jsp
          • tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/FetchScheduleEvent.java
          • tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java
          Show
          hudson Hudson added a comment - SUCCESS: Integrated in Tajo-master-build #1038 (See https://builds.apache.org/job/Tajo-master-build/1038/ ) TAJO-1950 : Query master uses too much memory during range shuffle. (jihoonson: rev 1f9ae1da0424731567cea18e975c47d4479b0ae9) tajo-core-tests/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java tajo-core-tests/src/test/java/org/apache/tajo/master/TestRepartitioner.java tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcher.java tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java tajo-core/src/main/proto/ResourceProtos.proto tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java CHANGES tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java tajo-core/src/main/resources/webapps/worker/task.jsp tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java tajo-core/src/main/java/org/apache/tajo/querymaster/FetchScheduleEvent.java tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java
          Hide
          hudson Hudson added a comment -

          FAILURE: Integrated in Tajo-0.11.1-build #140 (See https://builds.apache.org/job/Tajo-0.11.1-build/140/)
          TAJO-1950: Query master uses too much memory during range shuffle. (jihoonson: rev e3443c6df3ba6046b0e9b33e0da0ad27dae3ebbc)

          • tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java
          • tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
          • tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
          • tajo-core-tests/src/test/java/org/apache/tajo/master/TestRepartitioner.java
          • tajo-core/src/main/resources/webapps/worker/task.jsp
          • tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcher.java
          • tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
          • tajo-core-tests/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
          • tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java
          • CHANGES
          • tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java
          • tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
          • tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/FetchScheduleEvent.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java
          • tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java
          • tajo-core/src/main/proto/ResourceProtos.proto
          • tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
          • tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
          Show
          hudson Hudson added a comment - FAILURE: Integrated in Tajo-0.11.1-build #140 (See https://builds.apache.org/job/Tajo-0.11.1-build/140/ ) TAJO-1950 : Query master uses too much memory during range shuffle. (jihoonson: rev e3443c6df3ba6046b0e9b33e0da0ad27dae3ebbc) tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java tajo-core-tests/src/test/java/org/apache/tajo/master/TestRepartitioner.java tajo-core/src/main/resources/webapps/worker/task.jsp tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcher.java tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java tajo-core-tests/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java CHANGES tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java tajo-core/src/main/java/org/apache/tajo/querymaster/FetchScheduleEvent.java tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java tajo-core/src/main/proto/ResourceProtos.proto tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
          Hide
          jihoonson Jihoon Son added a comment -

          Committed to master and 0.11.1.

          Show
          jihoonson Jihoon Son added a comment - Committed to master and 0.11.1.
          Hide
          hudson Hudson added a comment -

          FAILURE: Integrated in Tajo-master-CODEGEN-build #640 (See https://builds.apache.org/job/Tajo-master-CODEGEN-build/640/)
          TAJO-1950: Query master uses too much memory during range shuffle. (jihoonson: rev 1f9ae1da0424731567cea18e975c47d4479b0ae9)

          • tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java
          • tajo-core/src/main/resources/webapps/worker/task.jsp
          • tajo-core/src/main/proto/ResourceProtos.proto
          • tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java
          • tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
          • tajo-core-tests/src/test/java/org/apache/tajo/master/TestRepartitioner.java
          • tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
          • tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java
          • tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
          • tajo-core-tests/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java
          • tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
          • tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java
          • tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java
          • tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcher.java
          • tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/FetchScheduleEvent.java
          • tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
          • CHANGES
          • tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
          • tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
          Show
          hudson Hudson added a comment - FAILURE: Integrated in Tajo-master-CODEGEN-build #640 (See https://builds.apache.org/job/Tajo-master-CODEGEN-build/640/ ) TAJO-1950 : Query master uses too much memory during range shuffle. (jihoonson: rev 1f9ae1da0424731567cea18e975c47d4479b0ae9) tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java tajo-core/src/main/resources/webapps/worker/task.jsp tajo-core/src/main/proto/ResourceProtos.proto tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java tajo-core-tests/src/test/java/org/apache/tajo/master/TestRepartitioner.java tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java tajo-core-tests/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java tajo-core-tests/src/test/java/org/apache/tajo/worker/TestFetcher.java tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java tajo-core/src/main/java/org/apache/tajo/querymaster/FetchScheduleEvent.java tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java CHANGES tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/tajo/pull/884

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/tajo/pull/884
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jihoonson commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-167075519

          @jinossy thanks for your comment. I'll commit shortly.
          Merry christmas!

          Show
          githubbot ASF GitHub Bot added a comment - Github user jihoonson commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-167075519 @jinossy thanks for your comment. I'll commit shortly. Merry christmas!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-167073515

          @jihoonson
          Thanks for your sharing. I guess, decreasing uri parameter will be better
          Ship it!

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-167073515 @jihoonson Thanks for your sharing. I guess, decreasing uri parameter will be better Ship it!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jihoonson commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-167070026

          I've decreased the default max url length to 1KB. Maybe we can find more optimized value of max url length and fetch timeout.

          Show
          githubbot ASF GitHub Bot added a comment - Github user jihoonson commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-167070026 I've decreased the default max url length to 1KB. Maybe we can find more optimized value of max url length and fetch timeout.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jihoonson commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-166503160

          Well, there remain some issues around fetch timeout. When I tested this patch against 10TB dataset, a lot of fetch timeouts occurred while transferring intermediate data between stages. The main reason looks that index lookup takes a lot of time (over 30 seconds with cache miss). So, I think the fundamental solution is to improve index search performance which need to be handled in another jira.
          So, I'd like to suggest to increase fetch timeout temporarily. With doubled timeout (that is 120 seconds), everything was ok against 10TB data.
          @hyunsik, @jinossy what do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user jihoonson commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-166503160 Well, there remain some issues around fetch timeout. When I tested this patch against 10TB dataset, a lot of fetch timeouts occurred while transferring intermediate data between stages. The main reason looks that index lookup takes a lot of time (over 30 seconds with cache miss). So, I think the fundamental solution is to improve index search performance which need to be handled in another jira. So, I'd like to suggest to increase fetch timeout temporarily. With doubled timeout (that is 120 seconds), everything was ok against 10TB data. @hyunsik, @jinossy what do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jihoonson commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-166245598

          @jinossy thanks for your review. I'm testing this patch against 10TB dataset. I'll share the result when it finishes and then commit this patch.

          Show
          githubbot ASF GitHub Bot added a comment - Github user jihoonson commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-166245598 @jinossy thanks for your review. I'm testing this patch against 10TB dataset. I'll share the result when it finishes and then commit this patch.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-165659875

          +1 LGTM!

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-165659875 +1 LGTM!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jihoonson commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-165392354

          @hyunsik and @jinossy, thanks for your comment. I make the max url length configurable. The default value is 2 KB.

          Show
          githubbot ASF GitHub Bot added a comment - Github user jihoonson commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-165392354 @hyunsik and @jinossy, thanks for your comment. I make the max url length configurable. The default value is 2 KB.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-165310118

          @hyunsik
          You're right. it can cause read timeout.

          @jihoonson
          Could you change to configurable value ?

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-165310118 @hyunsik You're right. it can cause read timeout. @jihoonson Could you change to configurable value ?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user hyunsik commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-165013250

          I have a concern about too long of MAXIMUM HTTP REQUEST. It may cause too large granularity for repeat fetch.

          Show
          githubbot ASF GitHub Bot added a comment - Github user hyunsik commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-165013250 I have a concern about too long of MAXIMUM HTTP REQUEST. It may cause too large granularity for repeat fetch.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-164643390

          Looks good to me!
          @hyunsik
          Have you any comments?

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-164643390 Looks good to me! @hyunsik Have you any comments?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on a diff in the pull request:

          https://github.com/apache/tajo/pull/884#discussion_r47465056

          — Diff: tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java —
          @@ -540,24 +555,21 @@ public boolean equals(Object obj)

          { return false; }
          • private FileFragment[] localizeFetchedData(File file, String name, TableMeta meta)
            + private FileFragment[] localizeFetchedData(String name)
            throws IOException {

          Configuration c = new Configuration(systemConf);
          c.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "file:///");
          FileSystem fs = FileSystem.get(c);

          • Path tablePath = new Path(file.getAbsolutePath());

          List<FileFragment> listTablets = new ArrayList<>();
          FileFragment tablet;

          • FileStatus[] fileLists = fs.listStatus(tablePath);
          • for (FileStatus f : fileLists) {
          • if (f.getLen() == 0) {
              • End diff –

          @jihoonson
          zero length chunk should be removed

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on a diff in the pull request: https://github.com/apache/tajo/pull/884#discussion_r47465056 — Diff: tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java — @@ -540,24 +555,21 @@ public boolean equals(Object obj) { return false; } private FileFragment[] localizeFetchedData(File file, String name, TableMeta meta) + private FileFragment[] localizeFetchedData(String name) throws IOException { Configuration c = new Configuration(systemConf); c.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "file:///"); FileSystem fs = FileSystem.get(c); Path tablePath = new Path(file.getAbsolutePath()); List<FileFragment> listTablets = new ArrayList<>(); FileFragment tablet; FileStatus[] fileLists = fs.listStatus(tablePath); for (FileStatus f : fileLists) { if (f.getLen() == 0) { End diff – @jihoonson zero length chunk should be removed
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jihoonson commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-164352045

          @jinossy thanks! I've updated my patch.

          Show
          githubbot ASF GitHub Bot added a comment - Github user jihoonson commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-164352045 @jinossy thanks! I've updated my patch.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on a diff in the pull request:

          https://github.com/apache/tajo/pull/884#discussion_r47459279

          — Diff: tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java —
          @@ -447,22 +499,35 @@ public void channelRegistered(ChannelHandlerContext ctx) throws Exception {
          public void channelRead0(ChannelHandlerContext ctx, FullHttpRequest request)
          throws Exception {

          • if (request.getMethod() != HttpMethod.GET) {
          • sendError(ctx, HttpResponseStatus.METHOD_NOT_ALLOWED);
            + if (request.getDecoderResult().isFailure()) { + LOG.error("Http decoding failed. ", request.getDecoderResult().cause()); + sendError(ctx, request.getDecoderResult().toString(), HttpResponseStatus.BAD_REQUEST); return; }
          • ProcessingStatus processingStatus = new ProcessingStatus(request.getUri().toString());
          • processingStatusMap.put(request.getUri().toString(), processingStatus);
            + if (request.getMethod() == HttpMethod.DELETE) {
            + HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.NO_CONTENT);
            + ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE);
            +
            + clearIndexCache(request.getUri());
              • End diff –

          You should add return state
          ```
          2015-12-14 12:33:27,993 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000001
          2015-12-14 12:33:27,995 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000004
          2015-12-14 12:33:27,999 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000003
          2015-12-14 12:33:28,059 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000008
          2015-12-14 12:33:28,060 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000007
          2015-12-14 12:33:28,074 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000002
          2015-12-14 12:33:28,111 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000009
          2015-12-14 12:33:28,158 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000011
          2015-12-14 12:33:28,181 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000012
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on a diff in the pull request: https://github.com/apache/tajo/pull/884#discussion_r47459279 — Diff: tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java — @@ -447,22 +499,35 @@ public void channelRegistered(ChannelHandlerContext ctx) throws Exception { public void channelRead0(ChannelHandlerContext ctx, FullHttpRequest request) throws Exception { if (request.getMethod() != HttpMethod.GET) { sendError(ctx, HttpResponseStatus.METHOD_NOT_ALLOWED); + if (request.getDecoderResult().isFailure()) { + LOG.error("Http decoding failed. ", request.getDecoderResult().cause()); + sendError(ctx, request.getDecoderResult().toString(), HttpResponseStatus.BAD_REQUEST); return; } ProcessingStatus processingStatus = new ProcessingStatus(request.getUri().toString()); processingStatusMap.put(request.getUri().toString(), processingStatus); + if (request.getMethod() == HttpMethod.DELETE) { + HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.NO_CONTENT); + ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); + + clearIndexCache(request.getUri()); End diff – You should add return state ``` 2015-12-14 12:33:27,993 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000001 2015-12-14 12:33:27,995 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000004 2015-12-14 12:33:27,999 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000003 2015-12-14 12:33:28,059 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000008 2015-12-14 12:33:28,060 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000007 2015-12-14 12:33:28,074 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000002 2015-12-14 12:33:28,111 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000009 2015-12-14 12:33:28,158 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000011 2015-12-14 12:33:28,181 ERROR: org.apache.tajo.pullserver.TajoPullServerService (channelRead0(523)) - Failed to decode uri eb_1450063997899_0015_000012 ```
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on a diff in the pull request:

          https://github.com/apache/tajo/pull/884#discussion_r47457822

          — Diff: tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java —
          @@ -191,10 +197,59 @@ public void stop(){
          }
          tasks.clear();
          taskHistories.clear();
          +
          + // Clear index cache in pull server
          + clearIndexCache();
          +
          resource.release();
          RpcClientManager.cleanup(queryMasterClient);
          }

          + private void clearIndexCache() {
          + if (executionBlockId.getId() > 1) {
          — End diff –

          Could you add some description?

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on a diff in the pull request: https://github.com/apache/tajo/pull/884#discussion_r47457822 — Diff: tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java — @@ -191,10 +197,59 @@ public void stop(){ } tasks.clear(); taskHistories.clear(); + + // Clear index cache in pull server + clearIndexCache(); + resource.release(); RpcClientManager.cleanup(queryMasterClient); } + private void clearIndexCache() { + if (executionBlockId.getId() > 1) { — End diff – Could you add some description?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on a diff in the pull request:

          https://github.com/apache/tajo/pull/884#discussion_r47457718

          — Diff: tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java —
          @@ -594,6 +668,36 @@ public void channelRead0(ChannelHandlerContext ctx, FullHttpRequest request)
          }
          }

          + private void clearIndexCache(String uri) throws IOException {
          — End diff –

          Could you add some description?

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on a diff in the pull request: https://github.com/apache/tajo/pull/884#discussion_r47457718 — Diff: tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java — @@ -594,6 +668,36 @@ public void channelRead0(ChannelHandlerContext ctx, FullHttpRequest request) } } + private void clearIndexCache(String uri) throws IOException { — End diff – Could you add some description?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on a diff in the pull request:

          https://github.com/apache/tajo/pull/884#discussion_r47457717

          — Diff: tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java —
          @@ -656,26 +760,61 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
          }
          }

          • public static FileChunk getFileChunks(Path outDir,
            + private static final ConcurrentHashMap<CacheKey, BSTIndexReader> waitForRemove = new ConcurrentHashMap<>();
            +
            + private static final RemovalListener<CacheKey, BSTIndexReader> removalListener = (removal) -> {
              • End diff –

          Could you add some description?

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on a diff in the pull request: https://github.com/apache/tajo/pull/884#discussion_r47457717 — Diff: tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java — @@ -656,26 +760,61 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) } } public static FileChunk getFileChunks(Path outDir, + private static final ConcurrentHashMap<CacheKey, BSTIndexReader> waitForRemove = new ConcurrentHashMap<>(); + + private static final RemovalListener<CacheKey, BSTIndexReader> removalListener = (removal) -> { End diff – Could you add some description?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on a diff in the pull request:

          https://github.com/apache/tajo/pull/884#discussion_r47457711

          — Diff: tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java —
          @@ -488,6 +497,18 @@ public BSTIndexReader(final Path fileName) throws IOException

          { open(); }

          + public void hold() {
          — End diff –

          How about changing the name to retain()?

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on a diff in the pull request: https://github.com/apache/tajo/pull/884#discussion_r47457711 — Diff: tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java — @@ -488,6 +497,18 @@ public BSTIndexReader(final Path fileName) throws IOException { open(); } + public void hold() { — End diff – How about changing the name to retain()?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on a diff in the pull request:

          https://github.com/apache/tajo/pull/884#discussion_r47457708

          — Diff: tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java —
          @@ -488,6 +497,18 @@ public BSTIndexReader(final Path fileName) throws IOException

          { open(); }

          + public void hold()

          { + REFERENCE_UPDATER.compareAndSet(this, referenceNum, referenceNum + 1); + }

          +
          + public void release() {
          — End diff –

          It seems to need the closing after release. if you add the description, it will be better

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on a diff in the pull request: https://github.com/apache/tajo/pull/884#discussion_r47457708 — Diff: tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java — @@ -488,6 +497,18 @@ public BSTIndexReader(final Path fileName) throws IOException { open(); } + public void hold() { + REFERENCE_UPDATER.compareAndSet(this, referenceNum, referenceNum + 1); + } + + public void release() { — End diff – It seems to need the closing after release. if you add the description, it will be better
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jihoonson commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-163894494

          This patch is ready for review.

          Show
          githubbot ASF GitHub Bot added a comment - Github user jihoonson commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-163894494 This patch is ready for review.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-163523745

          I see, please go ahead.

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-163523745 I see, please go ahead.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jihoonson commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-163515397

          @jinossy thanks for your review.
          Actually, I found more problems related to the http message length. I'll fix it soon.

          Show
          githubbot ASF GitHub Bot added a comment - Github user jihoonson commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-163515397 @jinossy thanks for your review. Actually, I found more problems related to the http message length. I'll fix it soon.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on a diff in the pull request:

          https://github.com/apache/tajo/pull/884#discussion_r47183366

          — Diff: tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java —
          @@ -191,10 +197,59 @@ public void stop(){
          }
          tasks.clear();
          taskHistories.clear();
          +
          + // Clear index cache in pull server
          + clearIndexCache();
          +
          resource.release();
          RpcClientManager.cleanup(queryMasterClient);
          }

          + private void clearIndexCache() {
          + if (executionBlockId.getId() > 1) {
          + Bootstrap bootstrap = new Bootstrap()
          + .group(NettyUtils.getSharedEventLoopGroup(NettyUtils.GROUP.FETCHER, 1))
          + .channel(NioSocketChannel.class)
          + .option(ChannelOption.ALLOCATOR, NettyUtils.ALLOCATOR)
          + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, 10 * 1000)
          + .option(ChannelOption.TCP_NODELAY, true);
          + ChannelInitializer<Channel> initializer = new ChannelInitializer<Channel>() {
          + @Override
          + protected void initChannel(Channel channel) throws Exception

          { + ChannelPipeline pipeline = channel.pipeline(); + pipeline.addLast("codec", new HttpClientCodec()); + }

          + };
          + bootstrap.handler(initializer);
          +
          + WorkerConnectionInfo connInfo = workerContext.getConnectionInfo();
          + ChannelFuture future = bootstrap.connect(new InetSocketAddress(connInfo.getHost(), connInfo.getPullServerPort()))
          + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE);
          +
          + try {
          + Channel channel = future.await().channel();
          + if (!future.isSuccess())

          { + // Upon failure to connect to pull server, cache clear message is just ignored. + LOG.warn(future.cause()); + return; + }

          +
          + ExecutionBlockId clearEbId = new ExecutionBlockId(executionBlockId.getQueryId(), executionBlockId.getId() - 1);
          + HttpRequest request = new DefaultHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.DELETE, clearEbId.toString());
          + request.headers().set(Names.HOST, connInfo.getHost());
          + request.headers().set(Names.CONNECTION, Values.CLOSE);
          + channel.writeAndFlush(request);
          + } catch (InterruptedException e)

          { + throw new RuntimeException(e); + }

          finally {
          + if (future != null && future.channel().isOpen()) {
          + // Close the channel to exit.
          + future.channel().closeFuture();
          — End diff –

          It should be `future.channel().close()`

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on a diff in the pull request: https://github.com/apache/tajo/pull/884#discussion_r47183366 — Diff: tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java — @@ -191,10 +197,59 @@ public void stop(){ } tasks.clear(); taskHistories.clear(); + + // Clear index cache in pull server + clearIndexCache(); + resource.release(); RpcClientManager.cleanup(queryMasterClient); } + private void clearIndexCache() { + if (executionBlockId.getId() > 1) { + Bootstrap bootstrap = new Bootstrap() + .group(NettyUtils.getSharedEventLoopGroup(NettyUtils.GROUP.FETCHER, 1)) + .channel(NioSocketChannel.class) + .option(ChannelOption.ALLOCATOR, NettyUtils.ALLOCATOR) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, 10 * 1000) + .option(ChannelOption.TCP_NODELAY, true); + ChannelInitializer<Channel> initializer = new ChannelInitializer<Channel>() { + @Override + protected void initChannel(Channel channel) throws Exception { + ChannelPipeline pipeline = channel.pipeline(); + pipeline.addLast("codec", new HttpClientCodec()); + } + }; + bootstrap.handler(initializer); + + WorkerConnectionInfo connInfo = workerContext.getConnectionInfo(); + ChannelFuture future = bootstrap.connect(new InetSocketAddress(connInfo.getHost(), connInfo.getPullServerPort())) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE); + + try { + Channel channel = future.await().channel(); + if (!future.isSuccess()) { + // Upon failure to connect to pull server, cache clear message is just ignored. + LOG.warn(future.cause()); + return; + } + + ExecutionBlockId clearEbId = new ExecutionBlockId(executionBlockId.getQueryId(), executionBlockId.getId() - 1); + HttpRequest request = new DefaultHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.DELETE, clearEbId.toString()); + request.headers().set(Names.HOST, connInfo.getHost()); + request.headers().set(Names.CONNECTION, Values.CLOSE); + channel.writeAndFlush(request); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } finally { + if (future != null && future.channel().isOpen()) { + // Close the channel to exit. + future.channel().closeFuture(); — End diff – It should be `future.channel().close()`
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jihoonson commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-163148708

          I've additionally added some codes to clean up index cache upon eb completion.

          Show
          githubbot ASF GitHub Bot added a comment - Github user jihoonson commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-163148708 I've additionally added some codes to clean up index cache upon eb completion.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jihoonson commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-162568053

          Hi, this patch is ready for review.
          Here are highlights of this patch.

          • To reduce the number of fetches created in Repartitioner, I've merged fetches with their pull server host name. As a result, the total number of fetches is ```H * R```, where ```H``` is the number of hosts and ```R``` is the number of ranges.
          • To reduce the size of fetches created in Repartitioner, I've changed to keep FetchProto instead of FetchImpl.
          • To reduce the number of index opens and closes, I've added an index reader cache to PullServerService.
          Show
          githubbot ASF GitHub Bot added a comment - Github user jihoonson commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-162568053 Hi, this patch is ready for review. Here are highlights of this patch. To reduce the number of fetches created in Repartitioner, I've merged fetches with their pull server host name. As a result, the total number of fetches is ```H * R```, where ```H``` is the number of hosts and ```R``` is the number of ranges. To reduce the size of fetches created in Repartitioner, I've changed to keep FetchProto instead of FetchImpl. To reduce the number of index opens and closes, I've added an index reader cache to PullServerService.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jihoonson commented on the pull request:

          https://github.com/apache/tajo/pull/884#issuecomment-160056176

          This patch is not ready for review. I need more tests including CI test.

          Show
          githubbot ASF GitHub Bot added a comment - Github user jihoonson commented on the pull request: https://github.com/apache/tajo/pull/884#issuecomment-160056176 This patch is not ready for review. I need more tests including CI test.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user jihoonson opened a pull request:

          https://github.com/apache/tajo/pull/884

          TAJO-1950: UniformRangePartition::partition() takes too long time when sorting large data

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/jihoonson/tajo-2 TAJO-1950

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/tajo/pull/884.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #884


          commit 9345f0e15734cac7dd24e107331240f9efbeba84
          Author: Jihoon Son <jihoonson@apache.org>
          Date: 2015-11-27T07:01:35Z

          TAJO-1950


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user jihoonson opened a pull request: https://github.com/apache/tajo/pull/884 TAJO-1950 : UniformRangePartition::partition() takes too long time when sorting large data You can merge this pull request into a Git repository by running: $ git pull https://github.com/jihoonson/tajo-2 TAJO-1950 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/tajo/pull/884.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #884 commit 9345f0e15734cac7dd24e107331240f9efbeba84 Author: Jihoon Son <jihoonson@apache.org> Date: 2015-11-27T07:01:35Z TAJO-1950
          Hide
          jihoonson Jihoon Son added a comment -

          I found several issues are related to this problem. I've opened some tickets for them.
          In this issue, I'll focus on reducing the memory usage of query master during range shuffle.

          Show
          jihoonson Jihoon Son added a comment - I found several issues are related to this problem. I've opened some tickets for them. In this issue, I'll focus on reducing the memory usage of query master during range shuffle.
          Hide
          jihoonson Jihoon Son added a comment -

          Thanks for your comment. I'm wondering 'too many fetch URIs' mean too many FetchImpl instances. If so, you are right. The second is the principal problem.

          As described in the document, the total size of created FetchImpl instances was about 80GB (130 for each fetch URL * 60000 FetchImpl instances * 10000 range partitions) when sorting 8TB data. Even if we can reduce the fetch URL length to 1 byte, the total size would be 570MB which will be increased with larger inputs.

          My proposal is to solve this problem, but I forgot to describe how it solves this problem in my proposal document.
          To reduce the number of FetchImpl instances, we need to figure out which hosts have data of which sort keys.
          To do so, we need to collect the information of data stored in each host. I think that using histogram is a good solution for it.

          Anyway, reducing the length of fetch URIs will be also helpful to alleviate the problem of too large amount of memory required for partitioning. It would be great if we handle it in another jira.

          Show
          jihoonson Jihoon Son added a comment - Thanks for your comment. I'm wondering 'too many fetch URIs' mean too many FetchImpl instances. If so, you are right. The second is the principal problem. As described in the document, the total size of created FetchImpl instances was about 80GB (130 for each fetch URL * 60000 FetchImpl instances * 10000 range partitions) when sorting 8TB data. Even if we can reduce the fetch URL length to 1 byte, the total size would be 570MB which will be increased with larger inputs. My proposal is to solve this problem, but I forgot to describe how it solves this problem in my proposal document. To reduce the number of FetchImpl instances, we need to figure out which hosts have data of which sort keys. To do so, we need to collect the information of data stored in each host. I think that using histogram is a good solution for it. Anyway, reducing the length of fetch URIs will be also helpful to alleviate the problem of too large amount of memory required for partitioning. It would be great if we handle it in another jira.
          Hide
          hyunsik Hyunsik Choi added a comment - - edited

          For sort, we firstly 1) aggregate a full range, 2) divide the range into N sub ranges, and then 3) generates tasks with fetch URIs. In the current implementation, I think there are two problems:

          • 1. uniform range partition over a full range without considering any data volume
          • 2. too many fetch URIs.

          In my opinion, the main reason why UniformRangePartition takes long time is the second problem. Your proposal seem to focus on the first problem.

          Show
          hyunsik Hyunsik Choi added a comment - - edited For sort, we firstly 1) aggregate a full range, 2) divide the range into N sub ranges, and then 3) generates tasks with fetch URIs. In the current implementation, I think there are two problems: 1. uniform range partition over a full range without considering any data volume 2. too many fetch URIs. In my opinion, the main reason why UniformRangePartition takes long time is the second problem. Your proposal seem to focus on the first problem.
          Hide
          jihoonson Jihoon Son added a comment -

          I've attached my proposal for this problem.

          Show
          jihoonson Jihoon Son added a comment - I've attached my proposal for this problem.

            People

            • Assignee:
              jihoonson Jihoon Son
              Reporter:
              jihoonson Jihoon Son
            • Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development