Details

    • Type: Sub-task
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 0.9.0
    • Component/s: Data Shuffle
    • Labels:
      None

      Description

      Currently PullServer runs on TajoWorker. If a TajoWorker starts Garbage Collector, all fetching tasks which run on other worker are paused during Garbage Collection. Running PullServer on a dedicated JVM process which separates from worker is helpful avoiding stop the all cluster by GC.
      PullServer uses small memory and doesn't share any information and status with TajoWorker.

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user babokim opened a pull request:

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

          TAJO-991: Running PullServer on a dedicated JVM process which separates from worker.

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

          $ git pull https://github.com/babokim/tajo TAJO-991

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

          https://github.com/apache/tajo/pull/107.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 #107


          commit 85a9918779428c25669b6d6a187f124a37dffb86
          Author: 김형준 <babokim@babokim-macbook-pro.local>
          Date: 2014-08-04T11:22:49Z

          TAJO-991: Running PullServer on a dedicated JVM process which separates from worker.

          commit 731d9160be1549fd8e13fecff321dfe3db604f2f
          Author: 김형준 <babokim@babokim-macbook-pro.local>
          Date: 2014-08-04T11:23:29Z

          Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/tajo

          commit 25f63a062840b540432eb50746f8f6eac6ebd4c6
          Author: 김형준 <babokim@babokim-macbook-pro.local>
          Date: 2014-08-04T13:38:26Z

          TAJO-991: Running PullServer on a dedicated JVM process which separates from worker.
          fix delete por file error in readPullServerPort.


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user babokim opened a pull request: https://github.com/apache/tajo/pull/107 TAJO-991 : Running PullServer on a dedicated JVM process which separates from worker. You can merge this pull request into a Git repository by running: $ git pull https://github.com/babokim/tajo TAJO-991 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/tajo/pull/107.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 #107 commit 85a9918779428c25669b6d6a187f124a37dffb86 Author: 김형준 <babokim@babokim-macbook-pro.local> Date: 2014-08-04T11:22:49Z TAJO-991 : Running PullServer on a dedicated JVM process which separates from worker. commit 731d9160be1549fd8e13fecff321dfe3db604f2f Author: 김형준 <babokim@babokim-macbook-pro.local> Date: 2014-08-04T11:23:29Z Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/tajo commit 25f63a062840b540432eb50746f8f6eac6ebd4c6 Author: 김형준 <babokim@babokim-macbook-pro.local> Date: 2014-08-04T13:38:26Z TAJO-991 : Running PullServer on a dedicated JVM process which separates from worker. fix delete por file error in readPullServerPort.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user hyunsik commented on the pull request:

          https://github.com/apache/tajo/pull/107#issuecomment-51735248

          Your patch is conflict against the latest revision. So, I've rebased it and sent a pull request to your working branch. Please review this.

          @jinossy This work is conflict against your recent work (TAJO-949). It would be great if you review Hyoungjun's work and my conflict resolution.

          Show
          githubbot ASF GitHub Bot added a comment - Github user hyunsik commented on the pull request: https://github.com/apache/tajo/pull/107#issuecomment-51735248 Your patch is conflict against the latest revision. So, I've rebased it and sent a pull request to your working branch. Please review this. @jinossy This work is conflict against your recent work ( TAJO-949 ). It would be great if you review Hyoungjun's work and my conflict resolution.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on the pull request:

          https://github.com/apache/tajo/pull/107#issuecomment-51737626

          OK, I will do review this. @babokim please merge this.

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on the pull request: https://github.com/apache/tajo/pull/107#issuecomment-51737626 OK, I will do review this. @babokim please merge this.
          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/107#discussion_r16338681

          — Diff: tajo-dist/src/main/conf/tajo-env.sh —
          @@ -68,4 +74,7 @@ export TAJO_WORKER_STANDBY_MODE=true

          1. It must be required to use HCatalogStore
          2. export HIVE_HOME=
            1. export HIVE_JDBC_DRIVER_DIR=
              \ No newline at end of file
              +# export HIVE_JDBC_DRIVER_DIR=
              +
              +# Tajo PullServer mode. the default mode is dedicated mode(dedicated or embedded)
              + export TAJO_PULLSERVER_MODE=dedicated
              • End diff –

          How about changing ‘TAJO_PULLSERVER_STANDALONE = true’ instead of dedicated/embeded ?

          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/107#discussion_r16338681 — Diff: tajo-dist/src/main/conf/tajo-env.sh — @@ -68,4 +74,7 @@ export TAJO_WORKER_STANDBY_MODE=true It must be required to use HCatalogStore export HIVE_HOME= export HIVE_JDBC_DRIVER_DIR= \ No newline at end of file +# export HIVE_JDBC_DRIVER_DIR= + +# Tajo PullServer mode. the default mode is dedicated mode(dedicated or embedded) + export TAJO_PULLSERVER_MODE=dedicated End diff – How about changing ‘TAJO_PULLSERVER_STANDALONE = true’ instead of dedicated/embeded ?
          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/107#discussion_r16338685

          — Diff: tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java —
          @@ -312,6 +377,63 @@ public ChannelPipeline getPipeline() throws Exception {
          }
          }

          +
          + Map<String, ProcessingStatus> processingStatusMap = new ConcurrentHashMap<String, ProcessingStatus>();
          +
          + public void completeFileChunk(FadvisedFileRegion filePart,
          + String requestUri,
          + long startTime) {
          + ProcessingStatus status = processingStatusMap.get(requestUri);
          + if (status != null)

          { + status.decrementRemainFiles(filePart, startTime); + }

          + }
          +
          + class ProcessingStatus {
          + String requestUri;
          + int numFiles;
          + AtomicInteger remainFiles;
          + long startTime;
          + long makeFileListTime;
          + long minTime = Long.MAX_VALUE;
          + long maxTime;
          + int numSlowFile;
          +
          + public ProcessingStatus(String requestUri)

          { + this.requestUri = requestUri; + this.startTime = System.currentTimeMillis(); + }

          +
          + public void setNumFiles(int numFiles)

          { + this.numFiles = numFiles; + this.remainFiles = new AtomicInteger(numFiles); + }

          + public void decrementRemainFiles(FadvisedFileRegion filePart, long fileStartTime) {
          + synchronized(remainFiles) {
          — End diff –

          remainFiles is AtomicInteger. please remove the synchronized block. also synchronized(processingStatusMap)

          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/107#discussion_r16338685 — Diff: tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java — @@ -312,6 +377,63 @@ public ChannelPipeline getPipeline() throws Exception { } } + + Map<String, ProcessingStatus> processingStatusMap = new ConcurrentHashMap<String, ProcessingStatus>(); + + public void completeFileChunk(FadvisedFileRegion filePart, + String requestUri, + long startTime) { + ProcessingStatus status = processingStatusMap.get(requestUri); + if (status != null) { + status.decrementRemainFiles(filePart, startTime); + } + } + + class ProcessingStatus { + String requestUri; + int numFiles; + AtomicInteger remainFiles; + long startTime; + long makeFileListTime; + long minTime = Long.MAX_VALUE; + long maxTime; + int numSlowFile; + + public ProcessingStatus(String requestUri) { + this.requestUri = requestUri; + this.startTime = System.currentTimeMillis(); + } + + public void setNumFiles(int numFiles) { + this.numFiles = numFiles; + this.remainFiles = new AtomicInteger(numFiles); + } + public void decrementRemainFiles(FadvisedFileRegion filePart, long fileStartTime) { + synchronized(remainFiles) { — End diff – remainFiles is AtomicInteger. please remove the synchronized block. also synchronized(processingStatusMap)
          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/107#discussion_r16338689

          — Diff: tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java —
          @@ -312,6 +377,63 @@ public ChannelPipeline getPipeline() throws Exception {
          }
          }

          +
          + Map<String, ProcessingStatus> processingStatusMap = new ConcurrentHashMap<String, ProcessingStatus>();
          +
          + public void completeFileChunk(FadvisedFileRegion filePart,
          + String requestUri,
          + long startTime) {
          + ProcessingStatus status = processingStatusMap.get(requestUri);
          + if (status != null)

          { + status.decrementRemainFiles(filePart, startTime); + }

          + }
          +
          + class ProcessingStatus {
          + String requestUri;
          + int numFiles;
          + AtomicInteger remainFiles;
          + long startTime;
          + long makeFileListTime;
          + long minTime = Long.MAX_VALUE;
          + long maxTime;
          + int numSlowFile;
          +
          + public ProcessingStatus(String requestUri)

          { + this.requestUri = requestUri; + this.startTime = System.currentTimeMillis(); + }

          +
          + public void setNumFiles(int numFiles)

          { + this.numFiles = numFiles; + this.remainFiles = new AtomicInteger(numFiles); + }

          + public void decrementRemainFiles(FadvisedFileRegion filePart, long fileStartTime) {
          + synchronized(remainFiles) {
          + long fileSendTime = System.currentTimeMillis() - fileStartTime;
          + if (fileSendTime > 20 * 1000)

          { + LOG.info("PullServer send too long time: filePos=" + filePart.getPosition() + ", fileLen=" + filePart.getCount()); + numSlowFile++; + }

          + if (fileSendTime > maxTime)

          { + maxTime = fileSendTime; + }

          + if (fileSendTime < minTime)

          { + minTime = fileSendTime; + }

          + int remain = remainFiles.decrementAndGet();
          + if (remain <= 0) {
          + synchronized(processingStatusMap) {
          — End diff –

          remove this block

          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/107#discussion_r16338689 — Diff: tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java — @@ -312,6 +377,63 @@ public ChannelPipeline getPipeline() throws Exception { } } + + Map<String, ProcessingStatus> processingStatusMap = new ConcurrentHashMap<String, ProcessingStatus>(); + + public void completeFileChunk(FadvisedFileRegion filePart, + String requestUri, + long startTime) { + ProcessingStatus status = processingStatusMap.get(requestUri); + if (status != null) { + status.decrementRemainFiles(filePart, startTime); + } + } + + class ProcessingStatus { + String requestUri; + int numFiles; + AtomicInteger remainFiles; + long startTime; + long makeFileListTime; + long minTime = Long.MAX_VALUE; + long maxTime; + int numSlowFile; + + public ProcessingStatus(String requestUri) { + this.requestUri = requestUri; + this.startTime = System.currentTimeMillis(); + } + + public void setNumFiles(int numFiles) { + this.numFiles = numFiles; + this.remainFiles = new AtomicInteger(numFiles); + } + public void decrementRemainFiles(FadvisedFileRegion filePart, long fileStartTime) { + synchronized(remainFiles) { + long fileSendTime = System.currentTimeMillis() - fileStartTime; + if (fileSendTime > 20 * 1000) { + LOG.info("PullServer send too long time: filePos=" + filePart.getPosition() + ", fileLen=" + filePart.getCount()); + numSlowFile++; + } + if (fileSendTime > maxTime) { + maxTime = fileSendTime; + } + if (fileSendTime < minTime) { + minTime = fileSendTime; + } + int remain = remainFiles.decrementAndGet(); + if (remain <= 0) { + synchronized(processingStatusMap) { — End diff – remove this block
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on the pull request:

          https://github.com/apache/tajo/pull/107#issuecomment-52456322

          @babokim
          Thank you for your contribution.
          I left some trivial comments.

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on the pull request: https://github.com/apache/tajo/pull/107#issuecomment-52456322 @babokim Thank you for your contribution. I left some trivial comments.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/tajo/pull/107#discussion_r16764866

          — Diff: tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java —
          @@ -312,6 +377,63 @@ public ChannelPipeline getPipeline() throws Exception {
          }
          }

          +
          + Map<String, ProcessingStatus> processingStatusMap = new ConcurrentHashMap<String, ProcessingStatus>();
          +
          + public void completeFileChunk(FadvisedFileRegion filePart,
          + String requestUri,
          + long startTime) {
          + ProcessingStatus status = processingStatusMap.get(requestUri);
          + if (status != null)

          { + status.decrementRemainFiles(filePart, startTime); + }

          + }
          +
          + class ProcessingStatus {
          + String requestUri;
          + int numFiles;
          + AtomicInteger remainFiles;
          + long startTime;
          + long makeFileListTime;
          + long minTime = Long.MAX_VALUE;
          + long maxTime;
          + int numSlowFile;
          +
          + public ProcessingStatus(String requestUri)

          { + this.requestUri = requestUri; + this.startTime = System.currentTimeMillis(); + }

          +
          + public void setNumFiles(int numFiles)

          { + this.numFiles = numFiles; + this.remainFiles = new AtomicInteger(numFiles); + }

          + public void decrementRemainFiles(FadvisedFileRegion filePart, long fileStartTime) {
          + synchronized(remainFiles) {
          — End diff –

          The maxTime and minTime variable is not atomic variable and this variable should be guaranteed thread safe.

          Show
          githubbot ASF GitHub Bot added a comment - Github user babokim commented on a diff in the pull request: https://github.com/apache/tajo/pull/107#discussion_r16764866 — Diff: tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java — @@ -312,6 +377,63 @@ public ChannelPipeline getPipeline() throws Exception { } } + + Map<String, ProcessingStatus> processingStatusMap = new ConcurrentHashMap<String, ProcessingStatus>(); + + public void completeFileChunk(FadvisedFileRegion filePart, + String requestUri, + long startTime) { + ProcessingStatus status = processingStatusMap.get(requestUri); + if (status != null) { + status.decrementRemainFiles(filePart, startTime); + } + } + + class ProcessingStatus { + String requestUri; + int numFiles; + AtomicInteger remainFiles; + long startTime; + long makeFileListTime; + long minTime = Long.MAX_VALUE; + long maxTime; + int numSlowFile; + + public ProcessingStatus(String requestUri) { + this.requestUri = requestUri; + this.startTime = System.currentTimeMillis(); + } + + public void setNumFiles(int numFiles) { + this.numFiles = numFiles; + this.remainFiles = new AtomicInteger(numFiles); + } + public void decrementRemainFiles(FadvisedFileRegion filePart, long fileStartTime) { + synchronized(remainFiles) { — End diff – The maxTime and minTime variable is not atomic variable and this variable should be guaranteed thread safe.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/tajo/pull/107#discussion_r16764955

          — Diff: tajo-dist/src/main/conf/tajo-env.sh —
          @@ -68,4 +74,7 @@ export TAJO_WORKER_STANDBY_MODE=true

          1. It must be required to use HCatalogStore
          2. export HIVE_HOME=
            1. export HIVE_JDBC_DRIVER_DIR=
              \ No newline at end of file
              +# export HIVE_JDBC_DRIVER_DIR=
              +
              +# Tajo PullServer mode. the default mode is dedicated mode(dedicated or embedded)
              + export TAJO_PULLSERVER_MODE=dedicated
              • End diff –

          Ok. I'll change.

          Show
          githubbot ASF GitHub Bot added a comment - Github user babokim commented on a diff in the pull request: https://github.com/apache/tajo/pull/107#discussion_r16764955 — Diff: tajo-dist/src/main/conf/tajo-env.sh — @@ -68,4 +74,7 @@ export TAJO_WORKER_STANDBY_MODE=true It must be required to use HCatalogStore export HIVE_HOME= export HIVE_JDBC_DRIVER_DIR= \ No newline at end of file +# export HIVE_JDBC_DRIVER_DIR= + +# Tajo PullServer mode. the default mode is dedicated mode(dedicated or embedded) + export TAJO_PULLSERVER_MODE=dedicated End diff – Ok. I'll change.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user babokim commented on the pull request:

          https://github.com/apache/tajo/pull/107#issuecomment-54012381

          I rebase and apply commented issues. Please review this patch.

          Show
          githubbot ASF GitHub Bot added a comment - Github user babokim commented on the pull request: https://github.com/apache/tajo/pull/107#issuecomment-54012381 I rebase and apply commented issues. Please review this 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/107#discussion_r16970239

          — Diff: tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java —
          @@ -622,6 +622,11 @@ public void handle(TaskEvent event) {
          }

          public void setIntermediateData(Collection<IntermediateEntry> partitions) {
          + for (IntermediateEntry i: partitions) {
          — End diff –

          looks like unused codes. please remove this

          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/107#discussion_r16970239 — Diff: tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java — @@ -622,6 +622,11 @@ public void handle(TaskEvent event) { } public void setIntermediateData(Collection<IntermediateEntry> partitions) { + for (IntermediateEntry i: partitions) { — End diff – looks like unused codes. please remove this
          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/107#discussion_r16970744

          — Diff: tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java —
          @@ -183,7 +181,13 @@ public void stop() {
          }

          protected void cleanupExecutionBlock(List<TajoIdProtos.ExecutionBlockIdProto> executionBlockIds) {

          • LOG.info("cleanup executionBlocks: " + executionBlockIds);
            + String cleanupMessage = "";
            + String prefix = "";
            + for (TajoIdProtos.ExecutionBlockIdProto eachEbId: executionBlockIds) {
            + cleanupMessage += prefix + (new ExecutionBlockId(eachEbId)).toString();
              • End diff –

          Could you change to StringBuilder ?

          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/107#discussion_r16970744 — Diff: tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java — @@ -183,7 +181,13 @@ public void stop() { } protected void cleanupExecutionBlock(List<TajoIdProtos.ExecutionBlockIdProto> executionBlockIds) { LOG.info("cleanup executionBlocks: " + executionBlockIds); + String cleanupMessage = ""; + String prefix = ""; + for (TajoIdProtos.ExecutionBlockIdProto eachEbId: executionBlockIds) { + cleanupMessage += prefix + (new ExecutionBlockId(eachEbId)).toString(); End diff – Could you change to StringBuilder ?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/tajo/pull/107#discussion_r16985288

          — Diff: tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java —
          @@ -183,7 +181,13 @@ public void stop() {
          }

          protected void cleanupExecutionBlock(List<TajoIdProtos.ExecutionBlockIdProto> executionBlockIds) {

          • LOG.info("cleanup executionBlocks: " + executionBlockIds);
            + String cleanupMessage = "";
            + String prefix = "";
            + for (TajoIdProtos.ExecutionBlockIdProto eachEbId: executionBlockIds) {
            + cleanupMessage += prefix + (new ExecutionBlockId(eachEbId)).toString();
              • End diff –

          Yes

          Show
          githubbot ASF GitHub Bot added a comment - Github user babokim commented on a diff in the pull request: https://github.com/apache/tajo/pull/107#discussion_r16985288 — Diff: tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java — @@ -183,7 +181,13 @@ public void stop() { } protected void cleanupExecutionBlock(List<TajoIdProtos.ExecutionBlockIdProto> executionBlockIds) { LOG.info("cleanup executionBlocks: " + executionBlockIds); + String cleanupMessage = ""; + String prefix = ""; + for (TajoIdProtos.ExecutionBlockIdProto eachEbId: executionBlockIds) { + cleanupMessage += prefix + (new ExecutionBlockId(eachEbId)).toString(); End diff – Yes
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/tajo/pull/107#discussion_r16985311

          — Diff: tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java —
          @@ -622,6 +622,11 @@ public void handle(TaskEvent event) {
          }

          public void setIntermediateData(Collection<IntermediateEntry> partitions) {
          + for (IntermediateEntry i: partitions) {
          — End diff –

          Ok, will be deleted.

          Show
          githubbot ASF GitHub Bot added a comment - Github user babokim commented on a diff in the pull request: https://github.com/apache/tajo/pull/107#discussion_r16985311 — Diff: tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java — @@ -622,6 +622,11 @@ public void handle(TaskEvent event) { } public void setIntermediateData(Collection<IntermediateEntry> partitions) { + for (IntermediateEntry i: partitions) { — End diff – Ok, will be deleted.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user jinossy commented on the pull request:

          https://github.com/apache/tajo/pull/107#issuecomment-54252062

          +1
          Looks great to me. Thank you for your contribution!

          Show
          githubbot ASF GitHub Bot added a comment - Github user jinossy commented on the pull request: https://github.com/apache/tajo/pull/107#issuecomment-54252062 +1 Looks great to me. Thank you for your contribution!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user babokim commented on the pull request:

          https://github.com/apache/tajo/pull/107#issuecomment-54262469

          Hi @jinossy.
          Thank you for your review.
          I've just committed it to the master branch.

          Show
          githubbot ASF GitHub Bot added a comment - Github user babokim commented on the pull request: https://github.com/apache/tajo/pull/107#issuecomment-54262469 Hi @jinossy. Thank you for your review. I've just committed it to the master branch.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

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

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

          SUCCESS: Integrated in Tajo-master-build #349 (See https://builds.apache.org/job/Tajo-master-build/349/)
          TAJO-991: Running PullServer on a dedicated JVM process which separates from worker. (babokim: rev 7c5ef87f6eedc13afb16311bbc3b27ea0d921eca)

          • tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java
          • tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java
          • tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
          • tajo-dist/src/main/bin/tajo
          • tajo-dist/src/main/conf/tajo-env.sh
          • tajo-yarn-pullserver/src/main/hadoop-2.4.0/org/apache/tajo/pullserver/listener/FileCloseListener.java
          • tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
          • tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
          • tajo-dist/src/main/bin/stop-tajo.sh
          • tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
          • tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
          • tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java
          • tajo-yarn-pullserver/src/main/hadoop-2.2.0/org/apache/tajo/pullserver/listener/FileCloseListener.java
          • CHANGES
          • tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
          • tajo-yarn-pullserver/src/main/hadoop-2.3.0/org/apache/tajo/pullserver/listener/FileCloseListener.java
          • tajo-dist/src/main/bin/start-tajo.sh
          Show
          hudson Hudson added a comment - SUCCESS: Integrated in Tajo-master-build #349 (See https://builds.apache.org/job/Tajo-master-build/349/ ) TAJO-991 : Running PullServer on a dedicated JVM process which separates from worker. (babokim: rev 7c5ef87f6eedc13afb16311bbc3b27ea0d921eca) tajo-core/src/main/java/org/apache/tajo/worker/Fetcher.java tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java tajo-dist/src/main/bin/tajo tajo-dist/src/main/conf/tajo-env.sh tajo-yarn-pullserver/src/main/hadoop-2.4.0/org/apache/tajo/pullserver/listener/FileCloseListener.java tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java tajo-dist/src/main/bin/stop-tajo.sh tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java tajo-yarn-pullserver/src/main/hadoop-2.2.0/org/apache/tajo/pullserver/listener/FileCloseListener.java CHANGES tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java tajo-yarn-pullserver/src/main/hadoop-2.3.0/org/apache/tajo/pullserver/listener/FileCloseListener.java tajo-dist/src/main/bin/start-tajo.sh
          Hide
          mhthanh Mai Hai Thanh added a comment -

          Hi Hyoungjun Kim,
          After TAJO-991, if I install Tajo by the command "mvn clean install -Pdist -Dtar", queries that require fetching data files will fail because Netty throws IllegalArgumentException. However, if I additionally specify "-Dhadoop.version=2.4.0", those queries will work. Do you know why ?

          If specifying "-Dhadoop.version=2.4.0" is mandatory, we should update the related documents.

          Show
          mhthanh Mai Hai Thanh added a comment - Hi Hyoungjun Kim , After TAJO-991 , if I install Tajo by the command "mvn clean install -Pdist -Dtar", queries that require fetching data files will fail because Netty throws IllegalArgumentException. However, if I additionally specify "-Dhadoop.version=2.4.0", those queries will work. Do you know why ? If specifying "-Dhadoop.version=2.4.0" is mandatory, we should update the related documents.
          Hide
          jhkim Jinho Kim added a comment -

          Hello Mai Hai Thanh,
          You are right. we should update the documents.
          Pullserver use the os cache in hadoop native. unfortunately hadoop codes was updated frequently.
          we need to specifying version for a while. and I will fix that if tajo binary is without specific version, this functions to be disabled.

          Show
          jhkim Jinho Kim added a comment - Hello Mai Hai Thanh , You are right. we should update the documents. Pullserver use the os cache in hadoop native. unfortunately hadoop codes was updated frequently. we need to specifying version for a while. and I will fix that if tajo binary is without specific version, this functions to be disabled.
          Hide
          jhkim Jinho Kim added a comment -

          Hyoungjun Kim
          This issue was committed. please close this issue

          Show
          jhkim Jinho Kim added a comment - Hyoungjun Kim This issue was committed. please close this issue
          Hide
          hyunsik Hyunsik Choi added a comment -

          Fixing it as resolved.

          Show
          hyunsik Hyunsik Choi added a comment - Fixing it as resolved.

            People

            • Assignee:
              hjkim Hyoungjun Kim
              Reporter:
              hjkim Hyoungjun Kim
            • Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development