Browse Source

MAPREDUCE-6069. Lint/style fixes and removal of unused code. Contributed by Todd Lipcon.

Todd Lipcon 11 năm trước cách đây
mục cha
commit
683987be7c
100 tập tin đã thay đổi với 394 bổ sung2012 xóa
  1. 1 0
      hadoop-mapreduce-project/CHANGES.MAPREDUCE-2841.txt
  2. 0 22
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/LICENSE
  3. 0 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt
  4. 0 13
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/Constants.java
  5. 0 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/DataReceiver.java
  6. 0 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/ICombineHandler.java
  7. 0 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/INativeHandler.java
  8. 0 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeBatchProcessor.java
  9. 6 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java
  10. 8 24
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java
  11. 4 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/Platform.java
  12. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPullee.java
  13. 0 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPuller.java
  14. 4 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPushee.java
  15. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPusher.java
  16. 10 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/CombinerHandler.java
  17. 0 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/IDataLoader.java
  18. 4 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/NativeCollectorOnlyHandler.java
  19. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/serde/BytesWritableSerializer.java
  20. 2 29
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/serde/IKVSerializer.java
  21. 4 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/serde/KVSerializer.java
  22. 7 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/serde/NativeSerialization.java
  23. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/BytesUtil.java
  24. 13 45
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/LocalJobOutputFiles.java
  25. 11 44
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/NativeTaskOutput.java
  26. 21 51
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/NativeTaskOutputFiles.java
  27. 11 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/ReadWriteBuffer.java
  28. 0 307
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/cityhash/city.cc
  29. 0 90
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/cityhash/city.h
  30. 0 137
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/NativeTask.h
  31. 2 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/BlockCodec.cc
  32. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/BlockCodec.h
  33. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/GzipCodec.cc
  34. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/GzipCodec.h
  35. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/Lz4Codec.cc
  36. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/Lz4Codec.h
  37. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/SnappyCodec.cc
  38. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/SnappyCodec.h
  39. 4 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/handler/AbstractMapHandler.cc
  40. 5 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/handler/BatchHandler.cc
  41. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/handler/CombineHandler.cc
  42. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/handler/CombineHandler.h
  43. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/handler/MCollectorOutputHandler.cc
  44. 2 114
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/BufferStream.cc
  45. 1 38
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/BufferStream.h
  46. 4 73
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Buffers.cc
  47. 3 60
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Buffers.h
  48. 0 73
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Combiner.cc
  49. 1 17
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Combiner.h
  50. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Compressions.cc
  51. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Compressions.h
  52. 12 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/FileSystem.cc
  53. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/FileSystem.h
  54. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/IFile.cc
  55. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/IFile.h
  56. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Iterator.cc
  57. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Log.cc
  58. 29 29
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MapOutputCollector.cc
  59. 8 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MapOutputCollector.h
  60. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MapOutputSpec.cc
  61. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MapOutputSpec.h
  62. 13 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MemoryBlock.cc
  63. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MemoryPool.h
  64. 2 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Merge.cc
  65. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Merge.h
  66. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MinHeap.h
  67. 3 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/NativeLibrary.cc
  68. 5 14
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/NativeObjectFactory.cc
  69. 16 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/NativeRuntimeJniImpl.cc
  70. 4 40
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/NativeTask.cc
  71. 12 13
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/PartitionBucket.cc
  72. 8 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/PartitionBucket.h
  73. 11 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/PartitionBucketIterator.cc
  74. 8 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/PartitionBucketIterator.h
  75. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Path.cc
  76. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/SpillInfo.cc
  77. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Streams.cc
  78. 1 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/TaskCounters.cc
  79. 0 50
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/TrackingCollector.h
  80. 4 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/commons.h
  81. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/jniutils.cc
  82. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Checksum.cc
  83. 0 24
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Hash.cc
  84. 0 73
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Hash.h
  85. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Random.cc
  86. 5 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/StringUtil.cc
  87. 4 110
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/SyncUtils.cc
  88. 0 228
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/SyncUtils.h
  89. 3 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Timer.cc
  90. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/WritableUtils.cc
  91. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/WritableUtils.h
  92. 5 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestCommand.cc
  93. 6 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestCompressions.cc
  94. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestConfig.cc
  95. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestCounter.cc
  96. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestFileSystem.cc
  97. 7 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestIFile.cc
  98. 14 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestMain.cc
  99. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestPrimitives.cc
  100. 8 25
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestSort.cc

+ 1 - 0
hadoop-mapreduce-project/CHANGES.MAPREDUCE-2841.txt

@@ -21,3 +21,4 @@ MAPREDUCE-6058. native-task: KVTest and LargeKVTest should check mr job is suces
 MAPREDUCE-6056. native-task: move system test working dir to target dir and cleanup test config xml files (Manu Zhang via bchang)
 MAPREDUCE-6055. native-task: findbugs, interface annotations, and other misc cleanup (todd)
 MAPREDUCE-6067. native-task: fix some counter issues (Binglin Chang)
+MAPREDUCE-6069. native-task: Lint/style fixes and removal of unused code (todd)

+ 0 - 22
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/LICENSE

@@ -204,28 +204,6 @@
 
 Licenses for third party projects used by this project:
 
-CityHash src/main/native/cityhash
----------------------------------------------------------------------
-// Copyright (c) 2011 Google, Inc.
-//
-// Permission is hereby granted, free of charge, to any person obtaining a copy
-// of this software and associated documentation files (the "Software"), to deal
-// in the Software without restriction, including without limitation the rights
-// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-// copies of the Software, and to permit persons to whom the Software is
-// furnished to do so, subject to the following conditions:
-//
-// The above copyright notice and this permission notice shall be included in
-// all copies or substantial portions of the Software.
-//
-// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-// THE SOFTWARE.
-
 GoogleTest src/main/native/gtest
 ---------------------------------------------------------------------
 Copyright 2008, Google Inc.

+ 0 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt

@@ -178,7 +178,6 @@ configure_file(main/native/test.sh test/test.sh)
 
 add_dual_library(nativetask
     ${CMAKE_BINARY_DIR}/lz4.c
-    ${D}/cityhash/city.cc
     ${D}/src/codec/BlockCodec.cc
     ${D}/src/codec/GzipCodec.cc
     ${D}/src/codec/Lz4Codec.cc
@@ -208,10 +207,8 @@ add_dual_library(nativetask
     ${D}/src/lib/SpillInfo.cc
     ${D}/src/lib/Path.cc
     ${D}/src/lib/Streams.cc
-    ${D}/src/lib/Combiner.cc
     ${D}/src/lib/TaskCounters.cc
     ${D}/src/util/Checksum.cc
-    ${D}/src/util/Hash.cc
     ${D}/src/util/Random.cc
     ${D}/src/util/StringUtil.cc
     ${D}/src/util/SyncUtils.cc
@@ -237,11 +234,8 @@ add_executable(nttest
     ${D}/test/lib/TestPartitionBucket.cc
     ${D}/test/lib/TestReadBuffer.cc
     ${D}/test/lib/TestReadWriteBuffer.cc
-    ${D}/test/lib/TestTrackingCollector.cc
     ${D}/test/util/TestChecksum.cc
-    ${D}/test/util/TestHash.cc
     ${D}/test/util/TestStringUtil.cc
-    ${D}/test/util/TestSyncUtils.cc
     ${D}/test/util/TestWritableUtils.cc
     ${D}/test/TestCommand.cc
     ${D}/test/TestConfig.cc

+ 0 - 13
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/Constants.java

@@ -25,22 +25,9 @@ public class Constants {
   public static final String MAP_SORT_CLASS = "map.sort.class";
   public static final String MAPRED_COMBINER_CLASS = "mapred.combiner.class";
 
-  public static final String MAPRED_MAPTASK_DELEGATOR_CLASS = "mapreduce.map.task.delegator.class";
-  public static final String MAPRED_REDUCETASK_DELEGATOR_CLASS = "mapreduce.reduce.task.delegator.class";
-  public static final String NATIVE_TASK_ENABLED = "native.task.enabled";
   public static final String NATIVE_LOG_DEVICE = "native.log.device";
   public static final String NATIVE_HADOOP_VERSION = "native.hadoop.version";
 
-  public static final String NATIVE_MAPPER_CLASS = "native.mapper.class";
-  public static final String NATIVE_REDUCER_CLASS = "native.reducer.class";
-  public static final String NATIVE_PARTITIONER_CLASS = "native.partitioner.class";
-  public static final String NATIVE_COMBINER_CLASS = "native.combiner.class";
-  public static final String NATIVE_INPUT_SPLIT = "native.input.split";
-
-  public static final String NATIVE_RECORDREADER_CLASS = "native.recordreader.class";
-  public static final String NATIVE_RECORDWRITER_CLASS = "native.recordwriter.class";
-  public static final String NATIVE_OUTPUT_FILE_NAME = "native.output.file.name";
-
   public static final String NATIVE_PROCESSOR_BUFFER_KB = "native.processor.buffer.kb";
   public static final int NATIVE_PROCESSOR_BUFFER_KB_DEFAULT = 64;
   public static final int NATIVE_ASYNC_PROCESSOR_BUFFER_KB_DEFAULT = 1024;

+ 0 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/DataReceiver.java

@@ -32,9 +32,6 @@ public interface DataReceiver {
   /**
    * Send a signal to the receiver that the data arrives.
    * The data is transferred in another band.
-   * 
-   * @return
-   * @throws IOException
    */
   public boolean receiveData() throws IOException;
 }

+ 0 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/ICombineHandler.java

@@ -29,7 +29,6 @@ public interface ICombineHandler {
 
   /**
    * run combiner
-   * @throws IOException
    */
   public void combine() throws IOException;
 
@@ -40,7 +39,6 @@ public interface ICombineHandler {
 
   /**
    * close handlers, buffer pullers and pushers
-   * @throws IOException
    */
   public void close() throws IOException;
 }

+ 0 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/INativeHandler.java

@@ -45,17 +45,9 @@ public interface INativeHandler extends NativeDataTarget, NativeDataSource {
 
   /**
    * call command to downstream
-   * 
-   * @param command
-   * @param parameter
-   * @return
-   * @throws IOException
    */
   public ReadWriteBuffer call(Command command, ReadWriteBuffer parameter) throws IOException;
 
-  /**
-   * @param handler
-   */
   void setCommandDispatcher(CommandDispatcher handler);
 
 }

+ 0 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeBatchProcessor.java

@@ -229,32 +229,21 @@ public class NativeBatchProcessor implements INativeHandler {
 
   /**
    * Let native side to process data in inputBuffer
-   * 
-   * @param handler
-   * @param length
    */
   private native void nativeProcessInput(long handler, int length);
 
   /**
    * Notice native side input is finished
-   * 
-   * @param handler
    */
   private native void nativeFinish(long handler);
 
   /**
    * Send control message to native side
-   * 
-   * @param cmd
-   *          command data
-   * @return return value
    */
   private native byte[] nativeCommand(long handler, int cmd, byte[] parameter);
 
   /**
    * Load data from native
-   * 
-   * @return
    */
   private native void nativeLoadData(long handler);
 

+ 6 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java

@@ -95,7 +95,8 @@ public class NativeMapOutputCollectorDelegator<K, V> implements MapOutputCollect
 
 
     if (!QuickSort.class.getName().equals(job.get(Constants.MAP_SORT_CLASS))) {
-      String message = "Native-Task doesn't support sort class " + job.get(Constants.MAP_SORT_CLASS);
+      String message = "Native-Task doesn't support sort class " +
+        job.get(Constants.MAP_SORT_CLASS);
       LOG.error(message);
       throw new InvalidJobConfException(message);
     }
@@ -115,8 +116,8 @@ public class NativeMapOutputCollectorDelegator<K, V> implements MapOutputCollect
         LOG.error(message);
         throw new InvalidJobConfException(message);
       } else if (!Platforms.support(keyCls.getName(), serializer, job)) {
-        String message = "Native output collector doesn't support this key, this key is not comparable in native "
-          + keyCls.getName();
+        String message = "Native output collector doesn't support this key, " +
+          "this key is not comparable in native: " + keyCls.getName();
         LOG.error(message);
         throw new InvalidJobConfException(message);
       }
@@ -144,7 +145,8 @@ public class NativeMapOutputCollectorDelegator<K, V> implements MapOutputCollect
       updater.start();
 
     } else {
-      String message = "Nativeruntime cannot be loaded, please check the libnativetask.so is in hadoop library dir";
+      String message = "NativeRuntime cannot be loaded, please check that " +
+        "libnativetask.so is in hadoop library dir";
       LOG.error(message);
       throw new InvalidJobConfException(message);
     }

+ 8 - 24
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java

@@ -79,9 +79,6 @@ public class NativeRuntime {
 
   /**
    * create native object We use it to create native handlers
-   * 
-   * @param clazz
-   * @return
    */
   public synchronized static long createNativeObject(String clazz) {
     assertNativeLibraryLoaded();
@@ -94,13 +91,11 @@ public class NativeRuntime {
 
   /**
    * Register a customized library
-   * 
-   * @param clazz
-   * @return
    */
   public synchronized static long registerLibrary(String libraryName, String clazz) {
     assertNativeLibraryLoaded();
-    final long ret = JNIRegisterModule(libraryName.getBytes(Charsets.UTF_8), clazz.getBytes(Charsets.UTF_8));
+    final long ret = JNIRegisterModule(libraryName.getBytes(Charsets.UTF_8),
+                                       clazz.getBytes(Charsets.UTF_8));
     if (ret != 0) {
       LOG.warn("Can't create NativeObject for class " + clazz + ", probably not exist.");
     }
@@ -117,9 +112,6 @@ public class NativeRuntime {
 
   /**
    * Get the status report from native space
-   * 
-   * @param reporter
-   * @throws IOException
    */
   public static void reportStatus(TaskReporter reporter) throws IOException {
     assertNativeLibraryLoaded();
@@ -164,40 +156,32 @@ public class NativeRuntime {
 
   /**
    * Config the native runtime with mapreduce job configurations.
-   * 
-   * @param configs
    */
   private native static void JNIConfigure(byte[][] configs);
 
   /**
    * create a native object in native space
-   * 
-   * @param clazz
-   * @return
    */
   private native static long JNICreateNativeObject(byte[] clazz);
 
   /**
    * create the default native object for certain type
-   * 
-   * @param type
-   * @return
    */
   @Deprecated
   private native static long JNICreateDefaultNativeObject(byte[] type);
 
   /**
    * destroy native object in native space
-   * 
-   * @param addr
    */
   private native static void JNIReleaseNativeObject(long addr);
 
   /**
-   * get status update from native side Encoding: progress:float status:Text Counter number: int the count of the
-   * counters Counters: array [group:Text, name:Text, incrCount:Long]
-   * 
-   * @return
+   * Get status update from native side
+   * Encoding:
+   *  progress:float
+   *  status:Text
+   *  number: int the count of the counters
+   *  Counters: array [group:Text, name:Text, incrCount:Long]
    */
   private native static byte[] JNIUpdateStatus();
 

+ 4 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/Platform.java

@@ -49,8 +49,6 @@ public abstract class Platform {
 
   /**
    * initialize a platform, where we should call registerKey
-   *
-   * @throws IOException
    */
   public abstract void init() throws IOException;
 
@@ -65,7 +63,6 @@ public abstract class Platform {
    *
    * @param keyClassName map out key class name
    * @param key          key serializer class
-   * @throws IOException
    */
   protected void registerKey(String keyClassName, Class<?> key) throws IOException {
     serialization.register(keyClassName, key);
@@ -92,12 +89,12 @@ public abstract class Platform {
   /**
    * whether it's the platform that has defined a custom Java comparator
    *
-   * NativeTask doesn't support custom Java comparator(set with mapreduce.job.output.key.comparator.class)
-   * but a platform (e.g Pig) could also set that conf and implement native comparators so
-   * we shouldn't bail out.
+   * NativeTask doesn't support custom Java comparators
+   * (set with mapreduce.job.output.key.comparator.class)
+   * but a platform (e.g Pig) could also set that conf and implement native
+   * comparators so we shouldn't bail out.
    *
    * @param keyComparator comparator set with mapreduce.job.output.key.comparator.class
-   * @return
    */
   protected abstract boolean define(Class<?> keyComparator);
 }

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPullee.java

@@ -46,7 +46,8 @@ public class BufferPullee<IK, IV> implements IDataLoader {
   private final NativeDataTarget target;
   private boolean closed = false;
   
-  public BufferPullee(Class<IK> iKClass, Class<IV> iVClass, RawKeyValueIterator rIter, NativeDataTarget target)
+  public BufferPullee(Class<IK> iKClass, Class<IV> iVClass,
+                      RawKeyValueIterator rIter, NativeDataTarget target)
       throws IOException {
     this.rIter = rIter;
     tmpInputKey = new SizedWritable<IK>(iKClass);

+ 0 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPuller.java

@@ -189,8 +189,6 @@ public class BufferPuller implements RawKeyValueIterator, DataReceiver {
   
   /**
    * Closes the iterator so that the underlying streams can be closed.
-   * 
-   * @throws IOException
    */
   @Override
   public void close() throws IOException {

+ 4 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPushee.java

@@ -51,7 +51,8 @@ public class BufferPushee<OK, OV> implements Closeable {
   private KVSerializer<OK, OV> deserializer;
   private boolean closed = false;
 
-  public BufferPushee(Class<OK> oKClass, Class<OV> oVClass, RecordWriter<OK, OV> writer) throws IOException {
+  public BufferPushee(Class<OK> oKClass, Class<OV> oVClass,
+                      RecordWriter<OK, OV> writer) throws IOException {
     tmpOutputKey = new SizedWritable<OK>(oKClass);
     tmpOutputValue = new SizedWritable<OV>(oVClass);
 
@@ -128,7 +129,8 @@ public class BufferPushee<OK, OV> implements Closeable {
       }
     }
     if (remain != totalRead) {
-      throw new IOException("We expect to read " + remain + ", but we actually read: " + totalRead);
+      throw new IOException("We expect to read " + remain +
+                            ", but we actually read: " + totalRead);
     }
     return true;
   }

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/BufferPusher.java

@@ -44,7 +44,8 @@ public class BufferPusher<K, V> implements OutputCollector<K, V> {
   IKVSerializer serializer;
   private boolean closed = false;
 
-  public BufferPusher(Class<K> iKClass, Class<V> iVClass, NativeDataTarget target) throws IOException {
+  public BufferPusher(Class<K> iKClass, Class<V> iVClass,
+                      NativeDataTarget target) throws IOException {
     tmpInputKey = new SizedWritable<K>(iKClass);
     tmpInputValue = new SizedWritable<V>(iVClass);
 

+ 10 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/CombinerHandler.java

@@ -49,7 +49,8 @@ class CombinerHandler<K, V> implements ICombineHandler, CommandDispatcher {
   private final BufferPusher<K, V> kvPusher;
   private boolean closed = false;
 
-  public static <K, V> ICombineHandler create(TaskContext context) throws IOException, ClassNotFoundException {
+  public static <K, V> ICombineHandler create(TaskContext context)
+    throws IOException, ClassNotFoundException {
     final JobConf conf = new JobConf(context.getConf());
     conf.set(Constants.SERIALIZATION_FRAMEWORK,
         String.valueOf(SerializationFramework.WRITABLE_SERIALIZATION.getType()));
@@ -66,11 +67,13 @@ class CombinerHandler<K, V> implements ICombineHandler, CommandDispatcher {
 
     final Counter combineInputCounter = context.getTaskReporter().getCounter(
         TaskCounter.COMBINE_INPUT_RECORDS);
-    
-    final CombinerRunner<K, V> combinerRunner = CombinerRunner.create(conf, context.getTaskAttemptId(),
+
+    final CombinerRunner<K, V> combinerRunner = CombinerRunner.create(
+        conf, context.getTaskAttemptId(),
         combineInputCounter, context.getTaskReporter(), null);
 
-    final INativeHandler nativeHandler = NativeBatchProcessor.create(NAME, conf, DataChannel.INOUT);
+    final INativeHandler nativeHandler = NativeBatchProcessor.create(
+      NAME, conf, DataChannel.INOUT);
     @SuppressWarnings("unchecked")
     final BufferPusher<K, V> pusher = new BufferPusher<K, V>((Class<K>)context.getInputKeyClass(),
         (Class<V>)context.getInputValueClass(),
@@ -79,8 +82,9 @@ class CombinerHandler<K, V> implements ICombineHandler, CommandDispatcher {
     return new CombinerHandler<K, V>(nativeHandler, combinerRunner, puller, pusher);
   }
 
-  public CombinerHandler(INativeHandler nativeHandler, CombinerRunner<K, V> combiner, BufferPuller puller,
-      BufferPusher<K, V> kvPusher) throws IOException {
+  public CombinerHandler(INativeHandler nativeHandler, CombinerRunner<K, V> combiner,
+                         BufferPuller puller, BufferPusher<K, V> kvPusher)
+    throws IOException {
     this.nativeHandler = nativeHandler;
     this.combinerRunner = combiner;
     this.puller = puller;

+ 0 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/IDataLoader.java

@@ -30,7 +30,6 @@ public interface IDataLoader {
 
   /**
    * @return size of data loaded
-   * @throws IOException
    */
   public int load() throws IOException;
 

+ 4 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/handlers/NativeCollectorOnlyHandler.java

@@ -63,7 +63,8 @@ public class NativeCollectorOnlyHandler<K, V> implements CommandDispatcher, Clos
   private final INativeHandler nativeHandler;
   private boolean closed = false;
 
-  public static <K, V> NativeCollectorOnlyHandler<K, V> create(TaskContext context) throws IOException {
+  public static <K, V> NativeCollectorOnlyHandler<K, V> create(TaskContext context)
+    throws IOException {
 
     
     ICombineHandler combinerHandler = null;
@@ -81,7 +82,8 @@ public class NativeCollectorOnlyHandler<K, V> implements CommandDispatcher, Clos
       LOG.info("[NativeCollectorOnlyHandler] combiner is not null");
     }
 
-    final INativeHandler nativeHandler = NativeBatchProcessor.create(NAME, context.getConf(), DataChannel.OUT);
+    final INativeHandler nativeHandler = NativeBatchProcessor.create(
+      NAME, context.getConf(), DataChannel.OUT);
     final BufferPusher<K, V> kvPusher = new BufferPusher<K, V>(
         (Class<K>)context.getOutputKeyClass(),
         (Class<V>)context.getOutputValueClass(),

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/serde/BytesWritableSerializer.java

@@ -27,7 +27,8 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.mapred.nativetask.INativeComparable;
 
 @InterfaceAudience.Private
-public class BytesWritableSerializer implements INativeComparable, INativeSerializer<BytesWritable> {
+public class BytesWritableSerializer
+  implements INativeComparable, INativeSerializer<BytesWritable> {
 
   @Override
   public int getLength(BytesWritable w) throws IOException {

+ 2 - 29
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/serde/IKVSerializer.java

@@ -33,43 +33,16 @@ public interface IKVSerializer {
 
   /**
    * update the length field of SizedWritable
-   * @param key
-   * @param value
-   * @throws IOException
    */
   public void updateLength(SizedWritable<?> key, SizedWritable<?> value) throws IOException;
 
-  /**
-   *
-   * @param out
-   * @param key
-   * @param value
-   * @return       bytes written
-   * @throws IOException
-   */
   public int serializeKV(DataOutputStream out, SizedWritable<?> key,
       SizedWritable<?> value) throws IOException;
 
-  /**
-   * serialize partitionId as well
-   * @param out
-   * @param partitionId
-   * @param key
-   * @param value
-   * @return
-   * @throws IOException
-   */
   public int serializePartitionKV(DataOutputStream out, int partitionId,
       SizedWritable<?> key, SizedWritable<?> value)
       throws IOException;
 
-  /**
-   *
-   * @param in
-   * @param key
-   * @param value
-   * @return      bytes read
-   * @throws IOException
-   */
-  public int deserializeKV(DataInputStream in, SizedWritable<?> key, SizedWritable<?> value) throws IOException;
+  public int deserializeKV(DataInputStream in, SizedWritable<?> key, SizedWritable<?> value)
+    throws IOException;
 }

+ 4 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/serde/KVSerializer.java

@@ -55,7 +55,8 @@ public class KVSerializer<K, V> implements IKVSerializer {
   }
 
   @Override
-  public int serializeKV(DataOutputStream out, SizedWritable<?> key, SizedWritable<?> value) throws IOException {
+  public int serializeKV(DataOutputStream out, SizedWritable<?> key, SizedWritable<?> value)
+    throws IOException {
     return serializePartitionKV(out, -1, key, value);
   }
 
@@ -64,7 +65,8 @@ public class KVSerializer<K, V> implements IKVSerializer {
       SizedWritable<?> key, SizedWritable<?> value)
       throws IOException {
 
-    if (key.length == SizedWritable.INVALID_LENGTH || value.length == SizedWritable.INVALID_LENGTH) {
+    if (key.length == SizedWritable.INVALID_LENGTH ||
+        value.length == SizedWritable.INVALID_LENGTH) {
       updateLength(key, value);
     }
 

+ 7 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/serde/NativeSerialization.java

@@ -27,7 +27,8 @@ import org.apache.hadoop.io.Writable;
 @InterfaceAudience.Private
 public class NativeSerialization {
 
-  private final ConcurrentHashMap<String, Class<?>> map = new ConcurrentHashMap<String, Class<?>>();
+  private final ConcurrentHashMap<String, Class<?>> map =
+    new ConcurrentHashMap<String, Class<?>>();
 
   public boolean accept(Class<?> c) {
     return Writable.class.isAssignableFrom(c);
@@ -40,7 +41,8 @@ public class NativeSerialization {
       return null;
     }
     if (!Writable.class.isAssignableFrom(c)) {
-      throw new IOException("Cannot serialize type " + c.getName() + ", we only accept subclass of Writable");
+      throw new IOException("Cannot serialize type " + c.getName() +
+                            ", we only accept subclass of Writable");
     }
     final String name = c.getName();
     final Class<?> serializer = map.get(name);
@@ -70,8 +72,9 @@ public class NativeSerialization {
       return;
     } else {
       if (!storedSerializer.getName().equals(serializer.getName())) {
-        throw new IOException("Error! Serializer already registered, exist: " + storedSerializer.getName() + ", new: "
-            + serializer.getName());
+        throw new IOException("Error! Serializer already registered, existing: " +
+                              storedSerializer.getName() + ", new: " +
+                              serializer.getName());
       }
     }
   }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/BytesUtil.java

@@ -97,7 +97,7 @@ public class BytesUtil {
    * Write a printable representation of a byte array.
    *
    * @param b byte array
-   * @return string
+   * @return the printable presentation
    * @see #toStringBinary(byte[], int, int)
    */
   public static String toStringBinary(final byte [] b) {

+ 13 - 45
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/LocalJobOutputFiles.java

@@ -45,9 +45,6 @@ public class LocalJobOutputFiles implements NativeTaskOutput {
 
   /**
    * Return the path to local map output file created earlier
-   * 
-   * @return path
-   * @throws IOException
    */
   public Path getOutputFile() throws IOException {
     String path = String.format(OUTPUT_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT);
@@ -57,10 +54,7 @@ public class LocalJobOutputFiles implements NativeTaskOutput {
   /**
    * Create a local map output file name.
    * 
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   * @param size the size of the file
    */
   public Path getOutputFileForWrite(long size) throws IOException {
     String path = String.format(OUTPUT_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT);
@@ -69,9 +63,6 @@ public class LocalJobOutputFiles implements NativeTaskOutput {
 
   /**
    * Return the path to a local map output index file created earlier
-   * 
-   * @return path
-   * @throws IOException
    */
   public Path getOutputIndexFile() throws IOException {
     String path = String.format(OUTPUT_FILE_INDEX_FORMAT_STRING, TASKTRACKER_OUTPUT);
@@ -81,10 +72,7 @@ public class LocalJobOutputFiles implements NativeTaskOutput {
   /**
    * Create a local map output index file name.
    * 
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   * @param size the size of the file
    */
   public Path getOutputIndexFileForWrite(long size) throws IOException {
     String path = String.format(OUTPUT_FILE_INDEX_FORMAT_STRING, TASKTRACKER_OUTPUT);
@@ -94,10 +82,7 @@ public class LocalJobOutputFiles implements NativeTaskOutput {
   /**
    * Return a local map spill file created earlier.
    * 
-   * @param spillNumber
-   *          the number
-   * @return path
-   * @throws IOException
+   * @param spillNumber the number
    */
   public Path getSpillFile(int spillNumber) throws IOException {
     String path = String.format(SPILL_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, spillNumber);
@@ -107,12 +92,8 @@ public class LocalJobOutputFiles implements NativeTaskOutput {
   /**
    * Create a local map spill file name.
    * 
-   * @param spillNumber
-   *          the number
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   * @param spillNumber the number
+   * @param size the size of the file
    */
   public Path getSpillFileForWrite(int spillNumber, long size) throws IOException {
     String path = String.format(SPILL_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, spillNumber);
@@ -122,10 +103,7 @@ public class LocalJobOutputFiles implements NativeTaskOutput {
   /**
    * Return a local map spill index file created earlier
    * 
-   * @param spillNumber
-   *          the number
-   * @return path
-   * @throws IOException
+   * @param spillNumber the number
    */
   public Path getSpillIndexFile(int spillNumber) throws IOException {
     String path = String
@@ -136,12 +114,8 @@ public class LocalJobOutputFiles implements NativeTaskOutput {
   /**
    * Create a local map spill index file name.
    * 
-   * @param spillNumber
-   *          the number
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   * @param spillNumber the number
+   * @param size the size of the file
    */
   public Path getSpillIndexFileForWrite(int spillNumber, long size) throws IOException {
     String path = String
@@ -152,10 +126,7 @@ public class LocalJobOutputFiles implements NativeTaskOutput {
   /**
    * Return a local reduce input file created earlier
    * 
-   * @param mapId
-   *          a map task id
-   * @return path
-   * @throws IOException
+   * @param mapId a map task id
    */
   public Path getInputFile(int mapId) throws IOException {
     return lDirAlloc.getLocalPathToRead(
@@ -166,14 +137,11 @@ public class LocalJobOutputFiles implements NativeTaskOutput {
   /**
    * Create a local reduce input file name.
    * 
-   * @param mapId
-   *          a map task id
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   * @param mapId a map task id
+   * @param size the size of the file
    */
-  public Path getInputFileForWrite(TaskID mapId, long size, Configuration conf) throws IOException {
+  public Path getInputFileForWrite(TaskID mapId, long size, Configuration conf)
+    throws IOException {
     return lDirAlloc.getLocalPathForWrite(
         String.format(REDUCE_INPUT_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, mapId.getId()), size,
         conf);

+ 11 - 44
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/NativeTaskOutput.java

@@ -32,103 +32,70 @@ public interface NativeTaskOutput {
 
   /**
    * Return the path to local map output file created earlier
-   * 
-   * @return path
-   * @throws IOException
    */
   public Path getOutputFile() throws IOException;
 
   /**
    * Create a local map output file name.
    * 
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   * @param size the size of the file
    */
   public Path getOutputFileForWrite(long size) throws IOException;
 
   /**
    * Return the path to a local map output index file created earlier
-   * 
-   * @return path
-   * @throws IOException
    */
   public Path getOutputIndexFile() throws IOException;
 
   /**
    * Create a local map output index file name.
    * 
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   * @param size the size of the file
    */
   public Path getOutputIndexFileForWrite(long size) throws IOException;
 
   /**
    * Return a local map spill file created earlier.
    * 
-   * @param spillNumber
-   *          the number
-   * @return path
-   * @throws IOException
+   * @param spillNumber the number
    */
   public Path getSpillFile(int spillNumber) throws IOException;
 
   /**
    * Create a local map spill file name.
    * 
-   * @param spillNumber
-   *          the number
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   * @param spillNumber the number
+   * @param size the size of the file
    */
   public Path getSpillFileForWrite(int spillNumber, long size) throws IOException;
 
   /**
    * Return a local map spill index file created earlier
    * 
-   * @param spillNumber
-   *          the number
-   * @return path
-   * @throws IOException
+   * @param spillNumber the number
    */
   public Path getSpillIndexFile(int spillNumber) throws IOException;
 
   /**
    * Create a local map spill index file name.
    * 
-   * @param spillNumber
-   *          the number
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+    r* @param spillNumber the number
+   * @param size the size of the file
    */
   public Path getSpillIndexFileForWrite(int spillNumber, long size) throws IOException;
 
   /**
    * Return a local reduce input file created earlier
    * 
-   * @param mapId
-   *          a map task id
-   * @return path
-   * @throws IOException
+   * @param mapId a map task id
    */
   public Path getInputFile(int mapId) throws IOException;
 
   /**
    * Create a local reduce input file name.
    * 
-   * @param mapId
-   *          a map task id
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   * @param mapId a map task id
+   * @param size the size of the file
    */
   public Path getInputFileForWrite(TaskID mapId, long size, Configuration conf) throws IOException;
 

+ 21 - 51
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/NativeTaskOutputFiles.java

@@ -28,11 +28,13 @@ import org.apache.hadoop.mapred.TaskID;
 
 /**
  * Manipulate the working area for the transient store for maps and reduces.
- * 
- * This class is used by map and reduce tasks to identify the directories that they need to write
- * to/read from for intermediate files. The callers of these methods are from child space and see
- * mapreduce.cluster.local.dir as taskTracker/jobCache/jobId/attemptId This class should not be used
- * from TaskTracker space.
+ *
+ * This class is used by map and reduce tasks to identify the directories that they need
+ * to write to/read from for intermediate files. The callers of these methods are from
+ * child space and see mapreduce.cluster.local.dir as
+ * taskTracker/jobCache/jobId/attemptId.
+ *
+ * This class should not be used from TaskTracker space.
  */
 @InterfaceAudience.Private
 public class NativeTaskOutputFiles implements NativeTaskOutput {
@@ -55,9 +57,6 @@ public class NativeTaskOutputFiles implements NativeTaskOutput {
 
   /**
    * Return the path to local map output file created earlier
-   * 
-   * @return path
-   * @throws IOException
    */
   public Path getOutputFile() throws IOException {
     String path = String.format(OUTPUT_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, id);
@@ -66,11 +65,8 @@ public class NativeTaskOutputFiles implements NativeTaskOutput {
 
   /**
    * Create a local map output file name.
-   * 
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   *
+   * @param size the size of the file
    */
   public Path getOutputFileForWrite(long size) throws IOException {
     String path = String.format(OUTPUT_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, id);
@@ -79,9 +75,6 @@ public class NativeTaskOutputFiles implements NativeTaskOutput {
 
   /**
    * Return the path to a local map output index file created earlier
-   * 
-   * @return path
-   * @throws IOException
    */
   public Path getOutputIndexFile() throws IOException {
     String path = String.format(OUTPUT_FILE_INDEX_FORMAT_STRING, TASKTRACKER_OUTPUT, id);
@@ -91,10 +84,7 @@ public class NativeTaskOutputFiles implements NativeTaskOutput {
   /**
    * Create a local map output index file name.
    * 
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   * @param size the size of the file
    */
   public Path getOutputIndexFileForWrite(long size) throws IOException {
     String path = String.format(OUTPUT_FILE_INDEX_FORMAT_STRING, TASKTRACKER_OUTPUT, id);
@@ -104,10 +94,7 @@ public class NativeTaskOutputFiles implements NativeTaskOutput {
   /**
    * Return a local map spill file created earlier.
    * 
-   * @param spillNumber
-   *          the number
-   * @return path
-   * @throws IOException
+   * @param spillNumber the number
    */
   public Path getSpillFile(int spillNumber) throws IOException {
     String path = String.format(SPILL_FILE_FORMAT_STRING, id, TASKTRACKER_OUTPUT, spillNumber);
@@ -117,12 +104,8 @@ public class NativeTaskOutputFiles implements NativeTaskOutput {
   /**
    * Create a local map spill file name.
    * 
-   * @param spillNumber
-   *          the number
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   * @param spillNumber the number
+   * @param size the size of the file
    */
   public Path getSpillFileForWrite(int spillNumber, long size) throws IOException {
     String path = String.format(SPILL_FILE_FORMAT_STRING, id, TASKTRACKER_OUTPUT, spillNumber);
@@ -132,10 +115,7 @@ public class NativeTaskOutputFiles implements NativeTaskOutput {
   /**
    * Return a local map spill index file created earlier
    * 
-   * @param spillNumber
-   *          the number
-   * @return path
-   * @throws IOException
+   * @param spillNumber the number
    */
   public Path getSpillIndexFile(int spillNumber) throws IOException {
     String path = String
@@ -146,12 +126,8 @@ public class NativeTaskOutputFiles implements NativeTaskOutput {
   /**
    * Create a local map spill index file name.
    * 
-   * @param spillNumber
-   *          the number
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   * @param spillNumber the number
+   * @param size the size of the file
    */
   public Path getSpillIndexFileForWrite(int spillNumber, long size) throws IOException {
     String path = String
@@ -162,10 +138,7 @@ public class NativeTaskOutputFiles implements NativeTaskOutput {
   /**
    * Return a local reduce input file created earlier
    * 
-   * @param mapId
-   *          a map task id
-   * @return path
-   * @throws IOException
+   * @param mapId a map task id
    */
   public Path getInputFile(int mapId) throws IOException {
     return lDirAlloc.getLocalPathToRead(
@@ -176,14 +149,11 @@ public class NativeTaskOutputFiles implements NativeTaskOutput {
   /**
    * Create a local reduce input file name.
    * 
-   * @param mapId
-   *          a map task id
-   * @param size
-   *          the size of the file
-   * @return path
-   * @throws IOException
+   * @param mapId a map task id
+   * @param size the size of the file
    */
-  public Path getInputFileForWrite(TaskID mapId, long size, Configuration conf) throws IOException {
+  public Path getInputFileForWrite(TaskID mapId, long size, Configuration conf)
+    throws IOException {
     return lDirAlloc.getLocalPathForWrite(
         String.format(REDUCE_INPUT_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, mapId.getId()), size,
         conf);

+ 11 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/ReadWriteBuffer.java

@@ -112,10 +112,15 @@ public class ReadWriteBuffer {
   }
 
   public long readLong() {
-    final long result = ((_buff[_readPoint + 0] & 255) << 0) + ((_buff[_readPoint + 1] & 255) << 8)
-        + ((_buff[_readPoint + 2] & 255) << 16) + ((long) (_buff[_readPoint + 3] & 255) << 24)
-        + ((long) (_buff[_readPoint + 4] & 255) << 32) + ((long) (_buff[_readPoint + 5] & 255) << 40)
-        + ((long) (_buff[_readPoint + 6] & 255) << 48) + (((long) _buff[_readPoint + 7] << 56));
+    final long result =
+      ((_buff[_readPoint + 0] & 255) << 0) +
+      ((_buff[_readPoint + 1] & 255) << 8) +
+      ((_buff[_readPoint + 2] & 255) << 16) +
+      ((long) (_buff[_readPoint + 3] & 255) << 24) +
+      ((long) (_buff[_readPoint + 4] & 255) << 32) +
+      ((long) (_buff[_readPoint + 5] & 255) << 40) +
+      ((long) (_buff[_readPoint + 6] & 255) << 48) +
+      (((long) _buff[_readPoint + 7] << 56));
 
     _readPoint += 8;
     return result;
@@ -144,7 +149,8 @@ public class ReadWriteBuffer {
     if (_buff.length - _writePoint >= toBeWritten) {
       return;
     }
-    final int newLength = (toBeWritten + _writePoint > CACHE_LINE_SIZE) ? (toBeWritten + _writePoint) : CACHE_LINE_SIZE;
+    final int newLength = (toBeWritten + _writePoint > CACHE_LINE_SIZE) ?
+      (toBeWritten + _writePoint) : CACHE_LINE_SIZE;
     final byte[] newBuff = new byte[newLength];
     System.arraycopy(_buff, 0, newBuff, 0, _writePoint);
     _buff = newBuff;

+ 0 - 307
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/cityhash/city.cc

@@ -1,307 +0,0 @@
-// Copyright (c) 2011 Google, Inc.
-//
-// Permission is hereby granted, free of charge, to any person obtaining a copy
-// of this software and associated documentation files (the "Software"), to deal
-// in the Software without restriction, including without limitation the rights
-// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-// copies of the Software, and to permit persons to whom the Software is
-// furnished to do so, subject to the following conditions:
-//
-// The above copyright notice and this permission notice shall be included in
-// all copies or substantial portions of the Software.
-//
-// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-// THE SOFTWARE.
-//
-// CityHash Version 1, by Geoff Pike and Jyrki Alakuijala
-//
-// This file provides CityHash64() and related functions.
-//
-// It's probably possible to create even faster hash functions by
-// writing a program that systematically explores some of the space of
-// possible hash functions, by using SIMD instructions, or by
-// compromising on hash quality.
-
-#include "city.h"
-
-#include <algorithm>
-
-using namespace std;
-
-#define UNALIGNED_LOAD64(p) (*(const uint64*)(p))
-#define UNALIGNED_LOAD32(p) (*(const uint32*)(p))
-
-#if !defined(LIKELY)
-#if defined(__GNUC__)
-#define LIKELY(x) (__builtin_expect(!!(x), 1))
-#else
-#define LIKELY(x) (x)
-#endif
-#endif
-
-// Some primes between 2^63 and 2^64 for various uses.
-static const uint64 k0 = 0xc3a5c85c97cb3127ULL;
-static const uint64 k1 = 0xb492b66fbe98f273ULL;
-static const uint64 k2 = 0x9ae16a3b2f90404fULL;
-static const uint64 k3 = 0xc949d7c7509e6557ULL;
-
-// Bitwise right rotate.  Normally this will compile to a single
-// instruction, especially if the shift is a manifest constant.
-static uint64 Rotate(uint64 val, int shift) {
-  // Avoid shifting by 64: doing so yields an undefined result.
-  return shift == 0 ? val : ((val >> shift) | (val << (64 - shift)));
-}
-
-// Equivalent to Rotate(), but requires the second arg to be non-zero.
-// On x86-64, and probably others, it's possible for this to compile
-// to a single instruction if both args are already in registers.
-static uint64 RotateByAtLeast1(uint64 val, int shift) {
-  return (val >> shift) | (val << (64 - shift));
-}
-
-static uint64 ShiftMix(uint64 val) {
-  return val ^ (val >> 47);
-}
-
-static uint64 HashLen16(uint64 u, uint64 v) {
-  return Hash128to64(uint128(u, v));
-}
-
-static uint64 HashLen0to16(const char *s, size_t len) {
-  if (len > 8) {
-    uint64 a = UNALIGNED_LOAD64(s);
-    uint64 b = UNALIGNED_LOAD64(s + len - 8);
-    return HashLen16(a, RotateByAtLeast1(b + len, len)) ^ b;
-  }
-  if (len >= 4) {
-    uint64 a = UNALIGNED_LOAD32(s);
-    return HashLen16(len + (a << 3), UNALIGNED_LOAD32(s + len - 4));
-  }
-  if (len > 0) {
-    uint8 a = s[0];
-    uint8 b = s[len >> 1];
-    uint8 c = s[len - 1];
-    uint32 y = static_cast<uint32>(a) + (static_cast<uint32>(b) << 8);
-    uint32 z = len + (static_cast<uint32>(c) << 2);
-    return ShiftMix(y * k2 ^ z * k3) * k2;
-  }
-  return k2;
-}
-
-// This probably works well for 16-byte strings as well, but it may be overkill
-// in that case.
-static uint64 HashLen17to32(const char *s, size_t len) {
-  uint64 a = UNALIGNED_LOAD64(s) * k1;
-  uint64 b = UNALIGNED_LOAD64(s + 8);
-  uint64 c = UNALIGNED_LOAD64(s + len - 8) * k2;
-  uint64 d = UNALIGNED_LOAD64(s + len - 16) * k0;
-  return HashLen16(Rotate(a - b, 43) + Rotate(c, 30) + d,
-                   a + Rotate(b ^ k3, 20) - c + len);
-}
-
-// Return a 16-byte hash for 48 bytes.  Quick and dirty.
-// Callers do best to use "random-looking" values for a and b.
-static pair<uint64, uint64> WeakHashLen32WithSeeds(
-    uint64 w, uint64 x, uint64 y, uint64 z, uint64 a, uint64 b) {
-  a += w;
-  b = Rotate(b + a + z, 21);
-  uint64 c = a;
-  a += x;
-  a += y;
-  b += Rotate(a, 44);
-  return make_pair(a + z, b + c);
-}
-
-// Return a 16-byte hash for s[0] ... s[31], a, and b.  Quick and dirty.
-static pair<uint64, uint64> WeakHashLen32WithSeeds(
-    const char* s, uint64 a, uint64 b) {
-  return WeakHashLen32WithSeeds(UNALIGNED_LOAD64(s),
-                                UNALIGNED_LOAD64(s + 8),
-                                UNALIGNED_LOAD64(s + 16),
-                                UNALIGNED_LOAD64(s + 24),
-                                a,
-                                b);
-}
-
-// Return an 8-byte hash for 33 to 64 bytes.
-static uint64 HashLen33to64(const char *s, size_t len) {
-  uint64 z = UNALIGNED_LOAD64(s + 24);
-  uint64 a = UNALIGNED_LOAD64(s) + (len + UNALIGNED_LOAD64(s + len - 16)) * k0;
-  uint64 b = Rotate(a + z, 52);
-  uint64 c = Rotate(a, 37);
-  a += UNALIGNED_LOAD64(s + 8);
-  c += Rotate(a, 7);
-  a += UNALIGNED_LOAD64(s + 16);
-  uint64 vf = a + z;
-  uint64 vs = b + Rotate(a, 31) + c;
-  a = UNALIGNED_LOAD64(s + 16) + UNALIGNED_LOAD64(s + len - 32);
-  z = UNALIGNED_LOAD64(s + len - 8);
-  b = Rotate(a + z, 52);
-  c = Rotate(a, 37);
-  a += UNALIGNED_LOAD64(s + len - 24);
-  c += Rotate(a, 7);
-  a += UNALIGNED_LOAD64(s + len - 16);
-  uint64 wf = a + z;
-  uint64 ws = b + Rotate(a, 31) + c;
-  uint64 r = ShiftMix((vf + ws) * k2 + (wf + vs) * k0);
-  return ShiftMix(r * k0 + vs) * k2;
-}
-
-uint64 CityHash64(const char *s, size_t len) {
-  if (len <= 32) {
-    if (len <= 16) {
-      return HashLen0to16(s, len);
-    } else {
-      return HashLen17to32(s, len);
-    }
-  } else if (len <= 64) {
-    return HashLen33to64(s, len);
-  }
-
-  // For strings over 64 bytes we hash the end first, and then as we
-  // loop we keep 56 bytes of state: v, w, x, y, and z.
-  uint64 x = UNALIGNED_LOAD64(s);
-  uint64 y = UNALIGNED_LOAD64(s + len - 16) ^ k1;
-  uint64 z = UNALIGNED_LOAD64(s + len - 56) ^ k0;
-  pair<uint64, uint64> v = WeakHashLen32WithSeeds(s + len - 64, len, y);
-  pair<uint64, uint64> w = WeakHashLen32WithSeeds(s + len - 32, len * k1, k0);
-  z += ShiftMix(v.second) * k1;
-  x = Rotate(z + x, 39) * k1;
-  y = Rotate(y, 33) * k1;
-
-  // Decrease len to the nearest multiple of 64, and operate on 64-byte chunks.
-  len = (len - 1) & ~static_cast<size_t>(63);
-  do {
-    x = Rotate(x + y + v.first + UNALIGNED_LOAD64(s + 16), 37) * k1;
-    y = Rotate(y + v.second + UNALIGNED_LOAD64(s + 48), 42) * k1;
-    x ^= w.second;
-    y ^= v.first;
-    z = Rotate(z ^ w.first, 33);
-    v = WeakHashLen32WithSeeds(s, v.second * k1, x + w.first);
-    w = WeakHashLen32WithSeeds(s + 32, z + w.second, y);
-    std::swap(z, x);
-    s += 64;
-    len -= 64;
-  } while (len != 0);
-  return HashLen16(HashLen16(v.first, w.first) + ShiftMix(y) * k1 + z,
-                   HashLen16(v.second, w.second) + x);
-}
-
-uint64 CityHash64WithSeed(const char *s, size_t len, uint64 seed) {
-  return CityHash64WithSeeds(s, len, k2, seed);
-}
-
-uint64 CityHash64WithSeeds(const char *s, size_t len,
-                           uint64 seed0, uint64 seed1) {
-  return HashLen16(CityHash64(s, len) - seed0, seed1);
-}
-
-// A subroutine for CityHash128().  Returns a decent 128-bit hash for strings
-// of any length representable in ssize_t.  Based on City and Murmur.
-static uint128 CityMurmur(const char *s, size_t len, uint128 seed) {
-  uint64 a = Uint128Low64(seed);
-  uint64 b = Uint128High64(seed);
-  uint64 c = 0;
-  uint64 d = 0;
-  ssize_t l = len - 16;
-  if (l <= 0) {  // len <= 16
-    c = b * k1 + HashLen0to16(s, len);
-    d = Rotate(a + (len >= 8 ? UNALIGNED_LOAD64(s) : c), 32);
-  } else {  // len > 16
-    c = HashLen16(UNALIGNED_LOAD64(s + len - 8) + k1, a);
-    d = HashLen16(b + len, c + UNALIGNED_LOAD64(s + len - 16));
-    a += d;
-    do {
-      a ^= ShiftMix(UNALIGNED_LOAD64(s) * k1) * k1;
-      a *= k1;
-      b ^= a;
-      c ^= ShiftMix(UNALIGNED_LOAD64(s + 8) * k1) * k1;
-      c *= k1;
-      d ^= c;
-      s += 16;
-      l -= 16;
-    } while (l > 0);
-  }
-  a = HashLen16(a, c);
-  b = HashLen16(d, b);
-  return uint128(a ^ b, HashLen16(b, a));
-}
-
-uint128 CityHash128WithSeed(const char *s, size_t len, uint128 seed) {
-  if (len < 128) {
-    return CityMurmur(s, len, seed);
-  }
-
-  // We expect len >= 128 to be the common case.  Keep 56 bytes of state:
-  // v, w, x, y, and z.
-  pair<uint64, uint64> v, w;
-  uint64 x = Uint128Low64(seed);
-  uint64 y = Uint128High64(seed);
-  uint64 z = len * k1;
-  v.first = Rotate(y ^ k1, 49) * k1 + UNALIGNED_LOAD64(s);
-  v.second = Rotate(v.first, 42) * k1 + UNALIGNED_LOAD64(s + 8);
-  w.first = Rotate(y + z, 35) * k1 + x;
-  w.second = Rotate(x + UNALIGNED_LOAD64(s + 88), 53) * k1;
-
-  // This is the same inner loop as CityHash64(), manually unrolled.
-  do {
-    x = Rotate(x + y + v.first + UNALIGNED_LOAD64(s + 16), 37) * k1;
-    y = Rotate(y + v.second + UNALIGNED_LOAD64(s + 48), 42) * k1;
-    x ^= w.second;
-    y ^= v.first;
-    z = Rotate(z ^ w.first, 33);
-    v = WeakHashLen32WithSeeds(s, v.second * k1, x + w.first);
-    w = WeakHashLen32WithSeeds(s + 32, z + w.second, y);
-    std::swap(z, x);
-    s += 64;
-    x = Rotate(x + y + v.first + UNALIGNED_LOAD64(s + 16), 37) * k1;
-    y = Rotate(y + v.second + UNALIGNED_LOAD64(s + 48), 42) * k1;
-    x ^= w.second;
-    y ^= v.first;
-    z = Rotate(z ^ w.first, 33);
-    v = WeakHashLen32WithSeeds(s, v.second * k1, x + w.first);
-    w = WeakHashLen32WithSeeds(s + 32, z + w.second, y);
-    std::swap(z, x);
-    s += 64;
-    len -= 128;
-  } while (LIKELY(len >= 128));
-  y += Rotate(w.first, 37) * k0 + z;
-  x += Rotate(v.first + z, 49) * k0;
-  // If 0 < len < 128, hash up to 4 chunks of 32 bytes each from the end of s.
-  for (size_t tail_done = 0; tail_done < len; ) {
-    tail_done += 32;
-    y = Rotate(y - x, 42) * k0 + v.second;
-    w.first += UNALIGNED_LOAD64(s + len - tail_done + 16);
-    x = Rotate(x, 49) * k0 + w.first;
-    w.first += v.first;
-    v = WeakHashLen32WithSeeds(s + len - tail_done, v.first, v.second);
-  }
-  // At this point our 48 bytes of state should contain more than
-  // enough information for a strong 128-bit hash.  We use two
-  // different 48-byte-to-8-byte hashes to get a 16-byte final result.
-  x = HashLen16(x, v.first);
-  y = HashLen16(y, w.first);
-  return uint128(HashLen16(x + v.second, w.second) + y,
-                 HashLen16(x + w.second, y + v.second));
-}
-
-uint128 CityHash128(const char *s, size_t len) {
-  if (len >= 16) {
-    return CityHash128WithSeed(s + 16,
-                               len - 16,
-                               uint128(UNALIGNED_LOAD64(s) ^ k3,
-                                       UNALIGNED_LOAD64(s + 8)));
-  } else if (len >= 8) {
-    return CityHash128WithSeed(NULL,
-                               0,
-                               uint128(UNALIGNED_LOAD64(s) ^ (len * k0),
-                                       UNALIGNED_LOAD64(s + len - 8) ^ k1));
-  } else {
-    return CityHash128WithSeed(s, len, uint128(k0, k1));
-  }
-}

+ 0 - 90
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/cityhash/city.h

@@ -1,90 +0,0 @@
-// Copyright (c) 2011 Google, Inc.
-//
-// Permission is hereby granted, free of charge, to any person obtaining a copy
-// of this software and associated documentation files (the "Software"), to deal
-// in the Software without restriction, including without limitation the rights
-// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-// copies of the Software, and to permit persons to whom the Software is
-// furnished to do so, subject to the following conditions:
-//
-// The above copyright notice and this permission notice shall be included in
-// all copies or substantial portions of the Software.
-//
-// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-// THE SOFTWARE.
-//
-// CityHash Version 1, by Geoff Pike and Jyrki Alakuijala
-//
-// This file provides a few functions for hashing strings. On x86-64
-// hardware in 2011, CityHash64() is faster than other high-quality
-// hash functions, such as Murmur.  This is largely due to higher
-// instruction-level parallelism.  CityHash64() and CityHash128() also perform
-// well on hash-quality tests.
-//
-// CityHash128() is optimized for relatively long strings and returns
-// a 128-bit hash.  For strings more than about 2000 bytes it can be
-// faster than CityHash64().
-//
-// Functions in the CityHash family are not suitable for cryptography.
-//
-// WARNING: This code has not been tested on big-endian platforms!
-// It is known to work well on little-endian platforms that have a small penalty
-// for unaligned reads, such as current Intel and AMD moderate-to-high-end CPUs.
-//
-// By the way, for some hash functions, given strings a and b, the hash
-// of a+b is easily derived from the hashes of a and b.  This property
-// doesn't hold for any hash functions in this file.
-
-#ifndef CITY_HASH_H_
-#define CITY_HASH_H_
-
-#include <stdlib.h>  // for size_t.
-#include <stdint.h>
-#include <utility>
-
-typedef uint8_t uint8;
-typedef uint32_t uint32;
-typedef uint64_t uint64;
-typedef std::pair<uint64, uint64> uint128;
-
-inline uint64 Uint128Low64(const uint128& x) { return x.first; }
-inline uint64 Uint128High64(const uint128& x) { return x.second; }
-
-// Hash function for a byte array.
-uint64 CityHash64(const char *buf, size_t len);
-
-// Hash function for a byte array.  For convenience, a 64-bit seed is also
-// hashed into the result.
-uint64 CityHash64WithSeed(const char *buf, size_t len, uint64 seed);
-
-// Hash function for a byte array.  For convenience, two seeds are also
-// hashed into the result.
-uint64 CityHash64WithSeeds(const char *buf, size_t len,
-                           uint64 seed0, uint64 seed1);
-
-// Hash function for a byte array.
-uint128 CityHash128(const char *s, size_t len);
-
-// Hash function for a byte array.  For convenience, a 128-bit seed is also
-// hashed into the result.
-uint128 CityHash128WithSeed(const char *s, size_t len, uint128 seed);
-
-// Hash 128 input bits down to 64 bits of output.
-// This is intended to be a reasonably good hash function.
-inline uint64 Hash128to64(const uint128& x) {
-  // Murmur-inspired hashing.
-  const uint64 kMul = 0x9ddfea08eb382d69ULL;
-  uint64 a = (Uint128Low64(x) ^ Uint128High64(x)) * kMul;
-  a ^= (a >> 47);
-  uint64 b = (Uint128High64(x) ^ a) * kMul;
-  b ^= (b >> 47);
-  b *= kMul;
-  return b;
-}
-
-#endif  // CITY_HASH_H_

+ 0 - 137
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/NativeTask.h

@@ -38,13 +38,6 @@ using std::pair;
 enum NativeObjectType {
   UnknownObjectType = 0,
   BatchHandlerType = 1,
-  MapperType = 2,
-  ReducerType = 3,
-  PartitionerType = 4,
-  CombinerType = 5,
-  FolderType = 6,
-  RecordReaderType = 7,
-  RecordWriterType = 8
 };
 
 /**
@@ -69,7 +62,6 @@ enum Endium {
 #define INPUT_LINE_KV_SEPERATOR "mapreduce.input.keyvaluelinerecordreader.key.value.separator"
 #define MAPRED_TEXTOUTPUT_FORMAT_SEPERATOR "mapreduce.output.textoutputformat.separator"
 #define MAPRED_WORK_OUT_DIR "mapreduce.task.output.dir"
-#define NATIVE_OUTPUT_FILE_NAME "native.output.file.name"
 #define MAPRED_COMPRESS_OUTPUT "mapreduce.output.fileoutputformat.compress"
 #define MAPRED_OUTPUT_COMPRESSION_CODEC "mapreduce.output.fileoutputformat.compress.codec"
 #define TOTAL_ORDER_PARTITIONER_PATH "total.order.partitioner.path"
@@ -386,32 +378,6 @@ public:
   virtual bool next(Buffer & key, Buffer & value) = 0;
 };
 
-class RecordReader : public KVIterator, public Configurable, public Progress {
-public:
-  virtual NativeObjectType type() {
-    return RecordReaderType;
-  }
-
-  virtual bool next(Buffer & key, Buffer & value) = 0;
-
-  virtual float getProgress() = 0;
-
-  virtual void close() = 0;
-};
-
-class RecordWriter : public Collector, public Configurable {
-public:
-  virtual NativeObjectType type() {
-    return RecordWriterType;
-  }
-
-  virtual void collect(const void * key, uint32_t keyLen, const void * value, uint32_t valueLen) {
-  }
-
-  virtual void close() {
-  }
-
-};
 
 class ProcessorBase : public Configurable {
 protected:
@@ -444,36 +410,6 @@ public:
   }
 };
 
-class Mapper : public ProcessorBase {
-public:
-  virtual NativeObjectType type() {
-    return MapperType;
-  }
-
-  /**
-   * Map interface, default IdenticalMapper
-   */
-  virtual void map(const char * key, uint32_t keyLen, const char * value, uint32_t valueLen) {
-    collect(key, keyLen, value, valueLen);
-  }
-};
-
-class Partitioner : public Configurable {
-public:
-  virtual NativeObjectType type() {
-    return PartitionerType;
-  }
-
-  /**
-   * Partition interface
-   * @param key key buffer
-   * @param keyLen key length, can be modified to smaller value
-   *               to truncate key
-   * @return partition number
-   */
-  virtual uint32_t getPartition(const char * key, uint32_t & keyLen, uint32_t numPartition);
-};
-
 enum KeyGroupIterState {
   SAME_KEY,
   NEW_KEY,
@@ -502,80 +438,7 @@ public:
   virtual const char * nextValue(uint32_t & len) = 0;
 };
 
-class Reducer : public ProcessorBase {
-public:
-  virtual NativeObjectType type() {
-    return ReducerType;
-  }
-
-  /**
-   * Reduce interface, default IdenticalReducer
-   */
-  virtual void reduce(KeyGroupIterator & input) {
-    const char * key;
-    const char * value;
-    uint32_t keyLen;
-    uint32_t valueLen;
-    key = input.getKey(keyLen);
-    while (NULL != (value = input.nextValue(valueLen))) {
-      collect(key, keyLen, value, valueLen);
-    }
-  }
-};
-
-/**
- * Folder API used for hashtable based aggregation
- * Folder will be used in this way:
- * on(key, value):
- *   state = hashtable.get(key)
- *   if state == None:
- *     size = size()
- *     if size == -1:
- *       state = init(null, -1)
- *     elif size > 0:
- *       state = fixallocator.get(key)
- *       init(state, size)
- *   folder(state, value, value.len)
- *
- * final():
- *   for k,state in hashtable:
- *     final(key, key.len, state)
- */
-class Folder : public ProcessorBase {
-public:
-  virtual NativeObjectType type() {
-    return FolderType;
-  }
-
-  /**
-   * Get aggregator state size
-   * @return state storage size
-   *         -1 size not fixed or unknown, default
-   *            e.g. list map tree
-   *         0  don't need to store state
-   *         >0  fixed sized state
-   *            e.g. int32 int64 float.
-   */
-  virtual int32_t size() {
-    return -1;
-  }
 
-  /**
-   * Create and/or init new state
-   */
-  virtual void * init(const char * key, uint32_t keyLen) {
-    return NULL;
-  }
-
-  /**
-   * Aggregation function
-   */
-  virtual void folder(void * dest, const char * value, uint32_t valueLen) {
-  }
-
-  virtual void final(const char * key, uint32_t keyLen, void * dest) {
-  }
-};
 
 enum KeyValueType {
   TextType = 0,

+ 2 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/BlockCodec.cc

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-#include "commons.h"
+#include "lib/commons.h"
 #include "NativeTask.h"
 #include "BlockCodec.h"
 
@@ -79,9 +79,8 @@ BlockDecompressStream::BlockDecompressStream(InputStream * stream, uint32_t buff
 }
 
 void BlockDecompressStream::init() {
-   _tempBufferSize = maxCompressedLength(_blockMax) + 8;
+  _tempBufferSize = maxCompressedLength(_blockMax) + 8;
   _tempBuffer = (char*)malloc(_tempBufferSize);
-
 }
 
 BlockDecompressStream::~BlockDecompressStream() {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/BlockCodec.h

@@ -19,7 +19,7 @@
 #ifndef BLOCKCODEC_H_
 #define BLOCKCODEC_H_
 
-#include "Compressions.h"
+#include "lib/Compressions.h"
 
 namespace NativeTask {
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/GzipCodec.cc

@@ -18,7 +18,7 @@
 
 #include <zconf.h>
 #include <zlib.h>
-#include "commons.h"
+#include "lib/commons.h"
 #include "GzipCodec.h"
 #include <iostream>
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/GzipCodec.h

@@ -19,7 +19,7 @@
 #ifndef GZIPCODEC_H_
 #define GZIPCODEC_H_
 
-#include "Compressions.h"
+#include "lib/Compressions.h"
 
 namespace NativeTask {
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/Lz4Codec.cc

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-#include "commons.h"
+#include "lib/commons.h"
 #include "lz4.h"
 #include "NativeTask.h"
 #include "Lz4Codec.h"

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/Lz4Codec.h

@@ -19,7 +19,7 @@
 #ifndef LZ4CODEC_H_
 #define LZ4CODEC_H_
 
-#include "Compressions.h"
+#include "lib/Compressions.h"
 #include "BlockCodec.h"
 
 namespace NativeTask {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/SnappyCodec.cc

@@ -19,7 +19,7 @@
 #include "config.h"
 
 #if defined HADOOP_SNAPPY_LIBRARY
-#include "commons.h"
+#include "lib/commons.h"
 #include "NativeTask.h"
 #include "SnappyCodec.h"
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/SnappyCodec.h

@@ -19,7 +19,7 @@
 #ifndef SNAPPYCODEC_H_
 #define SNAPPYCODEC_H_
 
-#include "Compressions.h"
+#include "lib/Compressions.h"
 #include "BlockCodec.h"
 
 namespace NativeTask {

+ 4 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/handler/AbstractMapHandler.cc

@@ -16,20 +16,19 @@
  * limitations under the License.
  */
 
-#include "commons.h"
+#include "lib/commons.h"
 #include "util/StringUtil.h"
 #include "MCollectorOutputHandler.h"
-#include "NativeObjectFactory.h"
-#include "MapOutputCollector.h"
+#include "lib/NativeObjectFactory.h"
+#include "lib/MapOutputCollector.h"
 #include "CombineHandler.h"
 
 using std::string;
 using std::vector;
 
 namespace NativeTask {
-
 const Command AbstractMapHandler::GET_OUTPUT_PATH(100, "GET_OUTPUT_PATH");
 const Command AbstractMapHandler::GET_OUTPUT_INDEX_PATH(101, "GET_OUTPUT_INDEX_PATH");
 const Command AbstractMapHandler::GET_SPILL_PATH(102, "GET_SPILL_PATH");
 const Command AbstractMapHandler::GET_COMBINE_HANDLER(103, "GET_COMBINE_HANDLER");
-}      //namespace
+} // namespace NativeTask

+ 5 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/handler/BatchHandler.cc

@@ -19,11 +19,11 @@
 #ifndef QUICK_BUILD
 #include "org_apache_hadoop_mapred_nativetask_NativeBatchProcessor.h"
 #endif
-#include "commons.h"
+#include "lib/commons.h"
 #include "jni_md.h"
-#include "jniutils.h"
+#include "lib/jniutils.h"
 #include "BatchHandler.h"
-#include "NativeObjectFactory.h"
+#include "lib/NativeObjectFactory.h"
 
 ///////////////////////////////////////////////////////////////
 // NativeBatchProcessor jni util methods
@@ -130,7 +130,8 @@ void BatchHandler::onSetup(Config * config, char * inputBuffer, uint32_t inputBu
     _out.reset(outputBuffer, outputBufferCapacity);
     _out.rewind(0, outputBufferCapacity);
 
-    LOG("[BatchHandler::onSetup] input Capacity %d, output capacity %d", inputBufferCapacity, _out.limit());
+    LOG("[BatchHandler::onSetup] input Capacity %d, output capacity %d",
+        inputBufferCapacity, _out.limit());
   }
   configure(_config);
 }

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/handler/CombineHandler.cc

@@ -208,7 +208,6 @@ void CombineHandler::write(char * buf, uint32_t length) {
     outputRecordCount++;
     remain -= kv->length();
     pos += kv->length();
-    ;
   }
 
   _combineOutputRecordCount += outputRecordCount;
@@ -242,7 +241,8 @@ void CombineHandler::combine(CombineContext type, KVIterator * kvIterator, IFile
   this->_writer = writer;
   call(COMBINE, NULL);
 
-  LOG("[CombineHandler] input Record Count: %d, input Bytes: %d, output Record Count: %d, output Bytes: %d",
+  LOG("[CombineHandler] input Record Count: %d, input Bytes: %d, "
+      "output Record Count: %d, output Bytes: %d",
       _combineInputRecordCount, _combineInputBytes,
       _combineOutputRecordCount, _combineOutputBytes);
   return;

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/handler/CombineHandler.h

@@ -18,7 +18,7 @@
 #ifndef _COMBINEHANDLER_H_
 #define _COMBINEHANDLER_H_
 
-#include "Combiner.h"
+#include "lib/Combiner.h"
 #include "BatchHandler.h"
 
 namespace NativeTask {

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/handler/MCollectorOutputHandler.cc

@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-#include "commons.h"
+#include "lib/commons.h"
 #include "util/StringUtil.h"
 #include "lib/TaskCounters.h"
 #include "MCollectorOutputHandler.h"
-#include "NativeObjectFactory.h"
-#include "MapOutputCollector.h"
+#include "lib/NativeObjectFactory.h"
+#include "lib/MapOutputCollector.h"
 #include "CombineHandler.h"
 
 using std::string;

+ 2 - 114
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/BufferStream.cc

@@ -16,123 +16,11 @@
  * limitations under the License.
  */
 
-#include "commons.h"
-#include "BufferStream.h"
+#include "lib/commons.h"
+#include "lib/BufferStream.h"
 
 namespace NativeTask {
 
-BufferedInputStream::BufferedInputStream(InputStream * stream, uint32_t bufferSize)
-    : FilterInputStream(stream), _buff(NULL), _position(0), _limit(0), _capacity(0) {
-  _buff = (char*)malloc(bufferSize);
-  if (NULL != _buff) {
-    LOG("[BuferStream] malloc failed when create BufferedInputStream with buffersize %u",
-        bufferSize);
-    _capacity = bufferSize;
-  }
-}
-
-BufferedInputStream::~BufferedInputStream() {
-  if (NULL != _buff) {
-    free(_buff);
-    _buff = NULL;
-    _position = 0;
-    _limit = 0;
-    _capacity = 0;
-  }
-}
-
-void BufferedInputStream::seek(uint64_t position) {
-  if (_limit - _position > 0) {
-    THROW_EXCEPTION(IOException, "temporary buffered data exists when fseek()");
-  }
-  _stream->seek(position);
-}
-
-uint64_t BufferedInputStream::tell() {
-  return _stream->tell() - (_limit - _position);
-}
-
-int32_t BufferedInputStream::read(void * buff, uint32_t length) {
-  uint32_t rest = _limit - _position;
-  if (rest > 0) {
-    // have some data in buffer, read from buffer
-    uint32_t cp = rest < length ? rest : length;
-    memcpy(buff, _buff + _position, cp);
-    _position += cp;
-    return cp;
-  } else if (length >= _capacity / 2) {
-    // dest buffer big enough, read to dest buffer directly
-    return _stream->read(buff, length);
-  } else {
-    // read to buffer first, then copy part of it to dest
-    _limit = 0;
-    do {
-      int32_t rd = _stream->read(_buff + _limit, _capacity - _limit);
-      if (rd <= 0) {
-        break;
-      }
-    } while (_limit < _capacity / 2);
-    if (_limit == 0) {
-      return -1;
-    }
-    uint32_t cp = _limit < length ? _limit : length;
-    memcpy(buff, _buff, cp);
-    _position = cp;
-    return cp;
-  }
-}
-
-/////////////////////////////////////////////////////////////////
-
-BufferedOutputStream::BufferedOutputStream(InputStream * stream, uint32_t bufferSize)
-    : FilterOutputStream(_stream), _buff(NULL), _position(0), _capacity(0) {
-  _buff = (char*)malloc(bufferSize + sizeof(uint64_t));
-  if (NULL != _buff) {
-    LOG("[BuferStream] malloc failed when create BufferedOutputStream with buffersize %u",
-        bufferSize);
-    _capacity = bufferSize;
-  }
-}
-
-BufferedOutputStream::~BufferedOutputStream() {
-  if (NULL != _buff) {
-    free(_buff);
-    _buff = NULL;
-    _position = 0;
-    _capacity = 0;
-  }
-}
-
-uint64_t BufferedOutputStream::tell() {
-  return _stream->tell() + _position;
-}
-
-void BufferedOutputStream::write(const void * buff, uint32_t length) {
-  if (length < _capacity / 2) {
-    uint32_t rest = _capacity - _position;
-    if (length < rest) {
-      simple_memcpy(_buff + _position, buff, length);
-      _position += length;
-    } else {
-      flush();
-      simple_memcpy(_buff, buff, length);
-      _position = length;
-    }
-  } else {
-    flush();
-    _stream->write(buff, length);
-  }
-}
-
-void BufferedOutputStream::flush() {
-  if (_position > 0) {
-    _stream->write(_buff, _position);
-    _position = 0;
-  }
-}
-
-///////////////////////////////////////////////////////////
-
 int32_t InputBuffer::read(void * buff, uint32_t length) {
   uint32_t rd = _capacity - _position < length ? _capacity - _position : length;
   if (rd > 0) {

+ 1 - 38
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/BufferStream.h

@@ -20,49 +20,12 @@
 #define BUFFERSTREAM_H_
 
 #include <string>
-#include "Streams.h"
+#include "lib/Streams.h"
 
 namespace NativeTask {
 
 using std::string;
 
-class BufferedInputStream : public FilterInputStream {
-protected:
-  char * _buff;
-  uint32_t _position;
-  uint32_t _limit;
-  uint32_t _capacity;
-public:
-  BufferedInputStream(InputStream * stream, uint32_t bufferSize = 64 * 1024);
-
-  virtual ~BufferedInputStream();
-
-  virtual void seek(uint64_t position);
-
-  virtual uint64_t tell();
-
-  virtual int32_t read(void * buff, uint32_t length);
-};
-
-class BufferedOutputStream : public FilterOutputStream {
-protected:
-  char * _buff;
-  uint32_t _position;
-  uint32_t _capacity;
-
-public:
-  BufferedOutputStream(InputStream * stream, uint32_t bufferSize = 64 * 1024);
-
-  virtual ~BufferedOutputStream();
-
-  virtual uint64_t tell();
-
-  virtual void write(const void * buff, uint32_t length);
-
-  virtual void flush();
-
-};
-
 class InputBuffer : public InputStream {
 protected:
   const char * _buff;

+ 4 - 73
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Buffers.cc

@@ -16,84 +16,15 @@
  * limitations under the License.
  */
 
-#include "commons.h"
+#include <string>
+
+#include "lib/commons.h"
 #include "util/StringUtil.h"
 #include "util/WritableUtils.h"
-#include "Buffers.h"
+#include "lib/Buffers.h"
 
 namespace NativeTask {
 
-DynamicBuffer::DynamicBuffer()
-    : _data(NULL), _capacity(0), _size(0), _used(0) {
-}
-
-DynamicBuffer::DynamicBuffer(uint32_t capacity)
-    : _data(NULL), _capacity(0), _size(0), _used(0) {
-  reserve(capacity);
-}
-
-DynamicBuffer::~DynamicBuffer() {
-  release();
-}
-
-void DynamicBuffer::release() {
-  if (_data != NULL) {
-    free(_data);
-    _data = NULL;
-    _capacity = 0;
-    _used = 0;
-  }
-}
-
-void DynamicBuffer::reserve(uint32_t capacity) {
-  if (_data != NULL) {
-    if (capacity > _capacity) {
-      char * newdata = (char*)realloc(_data, capacity);
-      if (newdata == NULL) {
-        THROW_EXCEPTION_EX(OutOfMemoryException, "DynamicBuffer reserve realloc %u failed",
-            capacity);
-      }
-      _data = newdata;
-      _capacity = capacity;
-    }
-    return;
-  }
-  release();
-  char * newdata = (char*)malloc(capacity);
-  if (newdata == NULL) {
-    THROW_EXCEPTION_EX(OutOfMemoryException, "DynamicBuffer reserve new %u failed", capacity);
-  }
-  _data = newdata;
-  _capacity = capacity;
-  _size = 0;
-  _used = 0;
-}
-
-int32_t DynamicBuffer::refill(InputStream * stream) {
-  if (_data == NULL || freeSpace() == 0) {
-    THROW_EXCEPTION(IOException, "refill DynamicBuffer failed, no space left");
-  }
-  int32_t rd = stream->read(_data + _size, freeSpace());
-  if (rd > 0) {
-    _size += rd;
-  }
-  return rd;
-}
-
-void DynamicBuffer::cleanUsed() {
-  if (_used > 0) {
-    uint32_t needToMove = _size - _used;
-    if (needToMove > 0) {
-      memmove(_data, _data + _used, needToMove);
-      _size = needToMove;
-    } else {
-      _size = 0;
-    }
-    _used = 0;
-  }
-}
-
-///////////////////////////////////////////////////////////
 
 ReadBuffer::ReadBuffer()
     : _buff(NULL), _remain(0), _size(0), _capacity(0), _stream(NULL), _source(NULL) {

+ 3 - 60
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Buffers.h

@@ -19,69 +19,12 @@
 #ifndef BUFFERS_H_
 #define BUFFERS_H_
 
-#include "Streams.h"
-#include "Compressions.h"
-#include "Constants.h"
+#include "lib/Streams.h"
+#include "lib/Compressions.h"
+#include "lib/Constants.h"
 
 namespace NativeTask {
 
-class DynamicBuffer {
-protected:
-  char * _data;
-  uint32_t _capacity;
-  uint32_t _size;
-  uint32_t _used;
-public:
-  DynamicBuffer();
-
-  DynamicBuffer(uint32_t capacity);
-
-  ~DynamicBuffer();
-
-  void reserve(uint32_t capacity);
-
-  void release();
-
-  uint32_t capacity() {
-    return _capacity;
-  }
-
-  char * data() {
-    return _data;
-  }
-
-  uint32_t size() {
-    return _size;
-  }
-
-  uint32_t used() {
-    return _used;
-  }
-
-  char * current() {
-    return _data + _used;
-  }
-
-  char * end() {
-    return _data + _size;
-  }
-
-  uint32_t remain() {
-    return _size - _used;
-  }
-
-  uint32_t freeSpace() {
-    return _capacity - _size;
-  }
-
-  void use(uint32_t count) {
-    _used += count;
-  }
-
-  void cleanUsed();
-
-  int32_t refill(InputStream * stream);
-};
 
 /**
  * A lightweight read buffer, act as buffered input stream

+ 0 - 73
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Combiner.cc

@@ -1,73 +0,0 @@
-/**
- * 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.
- */
-#include "Combiner.h"
-#include "StringUtil.h"
-
-namespace NativeTask {
-
-NativeCombineRunner::NativeCombineRunner(Config * config, ObjectCreatorFunc combinerCreator)
-    : _config(config), _combinerCreator(combinerCreator), _keyGroupCount(0) {
-  if (NULL == _combinerCreator) {
-    THROW_EXCEPTION_EX(UnsupportException, "Create combiner failed");
-  }
-}
-
-KeyGroupIterator * NativeCombineRunner::createKeyGroupIterator(KVIterator * iter) {
-  return new KeyGroupIteratorImpl(iter);
-}
-
-void NativeCombineRunner::combine(CombineContext context, KVIterator * iterator,
-    IFileWriter * writer) {
-  Configurable * combiner = (Configurable *)(_combinerCreator());
-  if (NULL != combiner) {
-    combiner->configure(_config);
-  }
-
-  NativeObjectType type = combiner->type();
-  switch (type) {
-  case MapperType: {
-    Mapper * mapper = (Mapper*)combiner;
-    mapper->setCollector(writer);
-
-    Buffer key;
-    Buffer value;
-    while (iterator->next(key, value)) {
-      mapper->map(key.data(), key.length(), value.data(), value.length());
-    }
-    mapper->close();
-    delete mapper;
-  }
-    break;
-  case ReducerType: {
-    Reducer * reducer = (Reducer*)combiner;
-    reducer->setCollector(writer);
-    KeyGroupIterator * kg = createKeyGroupIterator(iterator);
-    while (kg->nextKey()) {
-      _keyGroupCount++;
-      reducer->reduce(*kg);
-    }
-    reducer->close();
-    delete reducer;
-  }
-    break;
-  default:
-    THROW_EXCEPTION(UnsupportException, "Combiner type not support");
-  }
-}
-
-} /* namespace NativeTask */

+ 1 - 17
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Combiner.h

@@ -18,7 +18,7 @@
 #ifndef COMBINER_H_
 #define COMBINER_H_
 #include "commons.h"
-#include "IFile.h"
+#include "lib/IFile.h"
 
 namespace NativeTask {
 
@@ -66,21 +66,5 @@ public:
   }
 };
 
-class NativeCombineRunner : public ICombineRunner {
-private:
-  Config * _config;
-  ObjectCreatorFunc _combinerCreator;
-  uint32_t _keyGroupCount;
-
-public:
-  NativeCombineRunner(Config * config, ObjectCreatorFunc objectCreator);
-
-public:
-  void combine(CombineContext type, KVIterator * kvIterator, IFileWriter * writer);
-
-private:
-  KeyGroupIterator * createKeyGroupIterator(KVIterator * iter);
-};
-
 } /* namespace NativeTask */
 #endif /* COMBINER_H_ */

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Compressions.cc

@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-#include "commons.h"
+#include "lib/commons.h"
 #include "config.h"
-#include "SyncUtils.h"
-#include "Compressions.h"
+#include "lib/Compressions.h"
+#include "util/SyncUtils.h"
 #include "codec/GzipCodec.h"
 #include "codec/SnappyCodec.h"
 #include "codec/Lz4Codec.h"

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Compressions.h

@@ -21,7 +21,7 @@
 
 #include <string>
 #include <vector>
-#include "Streams.h"
+#include "lib/Streams.h"
 
 namespace NativeTask {
 

+ 12 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/FileSystem.cc

@@ -20,14 +20,14 @@
 #include <fcntl.h>
 #include <dirent.h>
 #include <sys/stat.h>
-#include "commons.h"
+#include "lib/commons.h"
 #include "util/StringUtil.h"
-#include "jniutils.h"
+#include "lib/jniutils.h"
 #include "NativeTask.h"
-#include "TaskCounters.h"
-#include "NativeObjectFactory.h"
-#include "Path.h"
-#include "FileSystem.h"
+#include "lib/TaskCounters.h"
+#include "lib/NativeObjectFactory.h"
+#include "lib/Path.h"
+#include "lib/FileSystem.h"
 
 namespace NativeTask {
 
@@ -122,14 +122,15 @@ void FileOutputStream::close() {
 /////////////////////////////////////////////////////////////
 
 class RawFileSystem : public FileSystem {
-protected:
+ protected:
   string getRealPath(const string & path) {
     if (StringUtil::StartsWith(path, "file:")) {
       return path.substr(5);
     }
     return path;
   }
-public:
+
+ public:
   InputStream * open(const string & path) {
     return new FileInputStream(getRealPath(path));
   }
@@ -206,7 +207,7 @@ public:
     struct stat sb;
 
     if (stat(np.c_str(), &sb) == 0) {
-      if (S_ISDIR (sb.st_mode) == 0) {
+      if (S_ISDIR(sb.st_mode) == 0) {
         return 1;
       }
       return 0;
@@ -226,7 +227,7 @@ public:
         if (mkdir(npath, nmode)) {
           return 1;
         }
-      } else if (S_ISDIR (sb.st_mode) == 0) {
+      } else if (S_ISDIR(sb.st_mode) == 0) {
         return 1;
       }
       *p++ = '/'; /* restore slash */
@@ -259,4 +260,4 @@ FileSystem & FileSystem::getLocal() {
   return RawFileSystemInstance;
 }
 
-} // namespace Hadoap
+} // namespace NativeTask

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/FileSystem.h

@@ -21,7 +21,7 @@
 
 #include <string>
 #include "NativeTask.h"
-#include "Streams.h"
+#include "lib/Streams.h"
 
 namespace NativeTask {
 

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/IFile.cc

@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-#include "commons.h"
+#include "lib/commons.h"
 #include "util/StringUtil.h"
-#include "IFile.h"
-#include "Compressions.h"
+#include "lib/IFile.h"
+#include "lib/Compressions.h"
 #include "lib/FileSystem.h"
 
 namespace NativeTask {

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/IFile.h

@@ -19,11 +19,11 @@
 #ifndef IFILE_H_
 #define IFILE_H_
 
-#include "Checksum.h"
-#include "Buffers.h"
-#include "WritableUtils.h"
-#include "SpillInfo.h"
-#include "MapOutputSpec.h"
+#include "util/Checksum.h"
+#include "lib/Buffers.h"
+#include "util/WritableUtils.h"
+#include "lib/SpillInfo.h"
+#include "lib/MapOutputSpec.h"
 
 namespace NativeTask {
 

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Iterator.cc

@@ -15,8 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include "Iterator.h"
-#include "commons.h"
+#include "lib/Iterator.h"
+#include "lib/commons.h"
 
 namespace NativeTask {
 
@@ -88,4 +88,4 @@ bool KeyGroupIteratorImpl::next() {
   return result;
 }
 
-} //namespace NativeTask
+} // namespace NativeTask

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Log.cc

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-#include "Log.h"
+#include "lib/Log.h"
 
 namespace NativeTask {
 

+ 29 - 29
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MapOutputCollector.cc

@@ -16,19 +16,21 @@
  * limitations under the License.
  */
 
-#include "commons.h"
+#include <string>
+
+#include "lib/commons.h"
 #include "util/Timer.h"
 #include "util/StringUtil.h"
-#include "FileSystem.h"
-#include "NativeObjectFactory.h"
-#include "MapOutputCollector.h"
-#include "Merge.h"
+#include "lib/FileSystem.h"
+#include "lib/NativeObjectFactory.h"
+#include "lib/MapOutputCollector.h"
+#include "lib/Merge.h"
 #include "NativeTask.h"
-#include "WritableUtils.h"
+#include "util/WritableUtils.h"
 #include "util/DualPivotQuickSort.h"
-#include "Combiner.h"
-#include "TaskCounters.h"
-#include "MinHeap.h"
+#include "lib/Combiner.h"
+#include "lib/TaskCounters.h"
+#include "lib/MinHeap.h"
 
 namespace NativeTask {
 
@@ -36,22 +38,18 @@ ICombineRunner * CombineRunnerWrapper::createCombiner() {
 
   ICombineRunner * combineRunner = NULL;
   if (NULL != _config->get(NATIVE_COMBINER)) {
-    const char * combinerClass = _config->get(NATIVE_COMBINER);
-    ObjectCreatorFunc objectCreater = NativeObjectFactory::GetObjectCreator(combinerClass);
-    if (NULL == objectCreater) {
-      THROW_EXCEPTION_EX(UnsupportException, "Combiner not found: %s", combinerClass);
-    } else {
-      LOG("[MapOutputCollector::configure] native combiner is enabled: %s", combinerClass);
-    }
-    combineRunner = new NativeCombineRunner(_config, objectCreater);
+    // Earlier versions of this code supported user-defined
+    // native Combiner implementations. This simplified version
+    // no longer supports it.
+    THROW_EXCEPTION_EX(UnsupportException, "Native Combiners not supported");
+  }
+
+  CombineHandler * javaCombiner = _spillOutput->getJavaCombineHandler();
+  if (NULL != javaCombiner) {
+    _isJavaCombiner = true;
+    combineRunner = (ICombineRunner *)javaCombiner;
   } else {
-    CombineHandler * javaCombiner = _spillOutput->getJavaCombineHandler();
-    if (NULL != javaCombiner) {
-      _isJavaCombiner = true;
-      combineRunner = (ICombineRunner *)javaCombiner;
-    } else {
-      LOG("[MapOutputCollector::getCombiner] cannot get combine handler from java");
-    }
+    LOG("[MapOutputCollector::getCombiner] cannot get combine handler from java");
   }
   return combineRunner;
 }
@@ -118,7 +116,7 @@ void MapOutputCollector::init(uint32_t defaultBlockSize, uint32_t memoryCapacity
 
   _pool->init(memoryCapacity);
 
-  //TODO: add support for customized comparator
+  // TODO: add support for customized comparator
   this->_keyComparator = keyComparator;
 
   _buckets = new PartitionBucket*[_numPartitions];
@@ -160,7 +158,8 @@ void MapOutputCollector::configure(Config * config) {
   uint32_t capacity = config->getInt(MAPRED_IO_SORT_MB, 300) * 1024 * 1024;
 
   uint32_t defaultBlockSize = getDefaultBlockSize(capacity, _numPartitions, maxBlockSize);
-  LOG("Native Total MemoryBlockPool: num_partitions %u, min_block_size %uK, max_block_size %uK, capacity %uM", _numPartitions, defaultBlockSize / 1024,
+  LOG("Native Total MemoryBlockPool: num_partitions %u, min_block_size %uK, "
+      "max_block_size %uK, capacity %uM", _numPartitions, defaultBlockSize / 1024,
       maxBlockSize / 1024, capacity / 1024 / 1024);
 
   ComparatorPtr comparator = getComparator(config, _spec);
@@ -179,7 +178,8 @@ void MapOutputCollector::configure(Config * config) {
 KVBuffer * MapOutputCollector::allocateKVBuffer(uint32_t partitionId, uint32_t kvlength) {
   PartitionBucket * partition = getPartition(partitionId);
   if (NULL == partition) {
-    THROW_EXCEPTION_EX(IOException, "Partition is NULL, partition_id: %d, num_partitions: %d", partitionId, _numPartitions);
+    THROW_EXCEPTION_EX(IOException, "Partition is NULL, partition_id: %d, num_partitions: %d",
+                       partitionId, _numPartitions);
   }
 
   KVBuffer * dest = partition->allocateKVBuffer(kvlength);
@@ -301,7 +301,7 @@ void MapOutputCollector::middleSpill(const std::string & spillOutput,
     info->path = spillOutput;
     uint64_t spillTime = timer.now() - timer.last() - metrics.sortTime;
 
-    const uint64_t M = 1000000; //million
+    const uint64_t M = 1000000; // million
     LOG("%s-spill: { id: %d, collect: %"PRIu64" ms, "
         "in-memory sort: %"PRIu64" ms, in-memory records: %"PRIu64", "
         "merge&spill: %"PRIu64" ms, uncompressed size: %"PRIu64", "
@@ -369,7 +369,7 @@ void MapOutputCollector::finalSpill(const std::string & filepath,
   uint64_t recordCount;
   writer->getStatistics(outputSize, realOutputSize, recordCount);
 
-  const uint64_t M = 1000000; //million
+  const uint64_t M = 1000000; // million
   LOG("Final-merge-spill: { id: %d, in-memory sort: %"PRIu64" ms, "
       "in-memory records: %"PRIu64", merge&spill: %"PRIu64" ms, "
       "records: %"PRIu64", uncompressed size: %"PRIu64", "

+ 8 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MapOutputCollector.h

@@ -20,14 +20,14 @@
 #define MAP_OUTPUT_COLLECTOR_H_
 
 #include "NativeTask.h"
-#include "MemoryPool.h"
-#include "Timer.h"
-#include "Buffers.h"
-#include "MapOutputSpec.h"
-#include "IFile.h"
-#include "SpillInfo.h"
-#include "Combiner.h"
-#include "PartitionBucket.h"
+#include "lib/MemoryPool.h"
+#include "util/Timer.h"
+#include "lib/Buffers.h"
+#include "lib/MapOutputSpec.h"
+#include "lib/IFile.h"
+#include "lib/SpillInfo.h"
+#include "lib/Combiner.h"
+#include "lib/PartitionBucket.h"
 #include "lib/SpillOutputService.h"
 
 namespace NativeTask {

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MapOutputSpec.cc

@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 
-#include "commons.h"
-#include "MapOutputSpec.h"
+#include "lib/commons.h"
+#include "lib/MapOutputSpec.h"
 #include "NativeTask.h"
 
 namespace NativeTask {

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MapOutputSpec.h

@@ -20,8 +20,8 @@
 #define MAPOUTPUTSPEC_H_
 
 #include <string>
-#include "Checksum.h"
-#include "WritableUtils.h"
+#include "util/Checksum.h"
+#include "util/WritableUtils.h"
 #include "NativeTask.h"
 
 namespace NativeTask {

+ 13 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MemoryBlock.cc

@@ -15,18 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+#include <algorithm>
+
 #include "NativeTask.h"
-#include "commons.h"
-#include "Timer.h"
-#include "Buffers.h"
-#include "MapOutputSpec.h"
-#include "IFile.h"
-#include "SpillInfo.h"
-#include "Combiner.h"
-#include "MapOutputSpec.h"
+#include "lib/commons.h"
+#include "util/Timer.h"
+#include "lib/Buffers.h"
+#include "lib/MapOutputSpec.h"
+#include "lib/IFile.h"
+#include "lib/SpillInfo.h"
+#include "lib/Combiner.h"
 
-#include "MemoryBlock.h"
-#include "MemoryPool.h"
+#include "lib/MemoryBlock.h"
+#include "lib/MemoryPool.h"
 #include "util/DualPivotQuickSort.h"
 
 namespace NativeTask {
@@ -62,4 +64,4 @@ void MemoryBlock::sort(SortAlgorithm type, ComparatorPtr comparator) {
   }
   _sorted = true;
 }
-} //namespace NativeTask
+} // namespace NativeTask

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MemoryPool.h

@@ -19,8 +19,8 @@
 #ifndef MEMORYPOOL_H_
 #define MEMORYPOOL_H_
 
-#include "Buffers.h"
-#include "MapOutputSpec.h"
+#include "lib/Buffers.h"
+#include "lib/MapOutputSpec.h"
 #include "NativeTask.h"
 #include "util/StringUtil.h"
 

+ 2 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Merge.cc

@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-#include "commons.h"
+#include "lib/commons.h"
 #include "util/Timer.h"
 #include "util/StringUtil.h"
-#include "Merge.h"
+#include "lib/Merge.h"
 #include "lib/FileSystem.h"
 
 namespace NativeTask {
@@ -34,7 +34,6 @@ Merger::Merger(IFileWriter * writer, Config * config, ComparatorPtr comparator,
     ICombineRunner * combineRunner)
     : _writer(writer), _config(config), _combineRunner(combineRunner), _first(true),
         _comparator(comparator) {
-
 }
 
 Merger::~Merger() {

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Merge.h

@@ -20,10 +20,10 @@
 #define MERGE_H_
 
 #include "NativeTask.h"
-#include "Buffers.h"
-#include "MapOutputCollector.h"
-#include "IFile.h"
-#include "MinHeap.h"
+#include "lib/Buffers.h"
+#include "lib/MapOutputCollector.h"
+#include "lib/IFile.h"
+#include "lib/MinHeap.h"
 
 namespace NativeTask {
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MinHeap.h

@@ -20,7 +20,7 @@
 #define MIN_HEAP_H_
 
 #include "NativeTask.h"
-#include "Buffers.h"
+#include "lib/Buffers.h"
 
 template<typename T, typename Compare>
 void heapify(T* first, int rt, int heap_len, Compare & Comp) {

+ 3 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/NativeLibrary.cc

@@ -18,9 +18,9 @@
 
 #include <dlfcn.h>
 
-#include "commons.h"
-#include "NativeObjectFactory.h"
-#include "NativeLibrary.h"
+#include "lib/commons.h"
+#include "lib/NativeObjectFactory.h"
+#include "lib/NativeLibrary.h"
 
 namespace NativeTask {
 
@@ -30,7 +30,6 @@ namespace NativeTask {
 
 NativeLibrary::NativeLibrary(const string & path, const string & name)
     : _path(path), _name(name), _getObjectCreatorFunc(NULL), _functionGetter(NULL) {
-
 }
 
 bool NativeLibrary::init() {

+ 5 - 14
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/NativeObjectFactory.cc

@@ -20,11 +20,11 @@
 #ifndef __CYGWIN__
 #include <execinfo.h>
 #endif
-#include "commons.h"
+#include "lib/commons.h"
 #include "NativeTask.h"
-#include "NativeObjectFactory.h"
-#include "NativeLibrary.h"
-#include "BufferStream.h"
+#include "lib/NativeObjectFactory.h"
+#include "lib/NativeLibrary.h"
+#include "lib/BufferStream.h"
 #include "util/StringUtil.h"
 #include "util/SyncUtils.h"
 #include "util/WritableUtils.h"
@@ -53,19 +53,10 @@ extern "C" void handler(int sig) {
 }
 
 DEFINE_NATIVE_LIBRARY(NativeTask) {
-  //signal(SIGSEGV, handler);
   REGISTER_CLASS(BatchHandler, NativeTask);
   REGISTER_CLASS(CombineHandler, NativeTask);
   REGISTER_CLASS(MCollectorOutputHandler, NativeTask);
-  REGISTER_CLASS(Mapper, NativeTask);
-  REGISTER_CLASS(Reducer, NativeTask);
-  REGISTER_CLASS(Partitioner, NativeTask);
-  REGISTER_CLASS(Folder, NativeTask);
   NativeObjectFactory::SetDefaultClass(BatchHandlerType, "NativeTask.BatchHandler");
-  NativeObjectFactory::SetDefaultClass(MapperType, "NativeTask.Mapper");
-  NativeObjectFactory::SetDefaultClass(ReducerType, "NativeTask.Reducer");
-  NativeObjectFactory::SetDefaultClass(PartitionerType, "NativeTask.Partitioner");
-  NativeObjectFactory::SetDefaultClass(FolderType, "NativeTask.Folder");
 }
 
 namespace NativeTask {
@@ -132,7 +123,7 @@ bool NativeObjectFactory::Init() {
     }
     const char * version = GetConfig().get(NATIVE_HADOOP_VERSION);
     LOG("[NativeObjectLibrary] NativeTask library initialized with hadoop %s",
-        version==NULL?"unkown":version);
+        version == NULL ? "unkown" : version);
   }
   return true;
 }

+ 16 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/NativeRuntimeJniImpl.cc

@@ -20,9 +20,9 @@
 #include "org_apache_hadoop_mapred_nativetask_NativeRuntime.h"
 #endif
 #include "config.h"
-#include "commons.h"
-#include "jniutils.h"
-#include "NativeObjectFactory.h"
+#include "lib/commons.h"
+#include "lib/jniutils.h"
+#include "lib/NativeObjectFactory.h"
 
 using namespace NativeTask;
 
@@ -36,8 +36,7 @@ using namespace NativeTask;
  * Signature: ([B)Z
  */
 JNIEXPORT jboolean JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_supportsCompressionCodec
-  (JNIEnv *jenv, jclass clazz, jbyteArray codec)
-{
+  (JNIEnv *jenv, jclass clazz, jbyteArray codec) {
   const std::string codecString = JNU_ByteArrayToString(jenv, codec);
   if ("org.apache.hadoop.io.compress.GzipCodec" == codecString) {
     return JNI_TRUE;
@@ -58,7 +57,8 @@ JNIEXPORT jboolean JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntim
  * Class:     org_apache_hadoop_mapred_nativetask_NativeRuntime
  * Method:    JNIRelease
  * Signature: ()V
- */JNIEXPORT void JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNIRelease(
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNIRelease(
     JNIEnv * jenv, jclass nativeRuntimeClass) {
   try {
     NativeTask::NativeObjectFactory::Release();
@@ -82,7 +82,8 @@ JNIEXPORT jboolean JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntim
  * Class:     org_apache_hadoop_mapred_nativetask_NativeRuntime
  * Method:    JNIConfigure
  * Signature: ([[B)V
- */JNIEXPORT void JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNIConfigure(
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNIConfigure(
     JNIEnv * jenv, jclass nativeRuntimeClass, jobjectArray configs) {
   try {
     NativeTask::Config & config = NativeTask::NativeObjectFactory::GetConfig();
@@ -139,7 +140,8 @@ jlong JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNICreateNa
  * Class:     org_apache_hadoop_mapred_nativetask_NativeRuntime
  * Method:    JNICreateDefaultNativeObject
  * Signature: ([B)J
- */JNIEXPORT jlong JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNICreateDefaultNativeObject(
+ */
+JNIEXPORT jlong JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNICreateDefaultNativeObject(
     JNIEnv * jenv, jclass nativeRuntimeClass, jbyteArray type) {
   try {
     std::string typeString = JNU_ByteArrayToString(jenv, type);
@@ -166,7 +168,8 @@ jlong JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNICreateNa
  * Class:     org_apache_hadoop_mapred_nativetask_NativeRuntime
  * Method:    JNIReleaseNativeObject
  * Signature: (J)V
- */JNIEXPORT void JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNIReleaseNativeObject(
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNIReleaseNativeObject(
     JNIEnv * jenv, jclass nativeRuntimeClass, jlong objectAddr) {
   try {
     NativeTask::NativeObject * nobj = ((NativeTask::NativeObject *)objectAddr);
@@ -196,7 +199,8 @@ jlong JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNICreateNa
  * Class:     org_apache_hadoop_mapred_nativetask_NativeRuntime
  * Method:    JNIRegisterModule
  * Signature: ([B[B)I
- */JNIEXPORT jint JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNIRegisterModule(
+ */
+JNIEXPORT jint JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNIRegisterModule(
     JNIEnv * jenv, jclass nativeRuntimeClass, jbyteArray modulePath, jbyteArray moduleName) {
   try {
     std::string pathString = JNU_ByteArrayToString(jenv, modulePath);
@@ -225,7 +229,8 @@ jlong JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNICreateNa
  * Class:     org_apache_hadoop_mapred_nativetask_NativeRuntime
  * Method:    JNIUpdateStatus
  * Signature: ()[B
- */JNIEXPORT jbyteArray JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNIUpdateStatus(
+ */
+JNIEXPORT jbyteArray JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeRuntime_JNIUpdateStatus(
     JNIEnv * jenv, jclass nativeRuntimeClass) {
   try {
     std::string statusData;

+ 4 - 40
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/NativeTask.cc

@@ -18,11 +18,10 @@
 #ifndef __CYGWIN__
 #include <execinfo.h>
 #endif
-#include "commons.h"
-#include "util/Hash.h"
+#include "lib/commons.h"
 #include "util/StringUtil.h"
 #include "NativeTask.h"
-#include "NativeObjectFactory.h"
+#include "lib/NativeObjectFactory.h"
 
 namespace NativeTask {
 
@@ -34,20 +33,6 @@ const string NativeObjectTypeToString(NativeObjectType type) {
   switch (type) {
   case BatchHandlerType:
     return string("BatchHandlerType");
-  case MapperType:
-    return string("MapperType");
-  case ReducerType:
-    return string("ReducerType");
-  case PartitionerType:
-    return string("PartitionerType");
-  case CombinerType:
-    return string("CombinerType");
-  case FolderType:
-    return string("FolderType");
-  case RecordReaderType:
-    return string("RecordReaderType");
-  case RecordWriterType:
-    return string("RecordWriterType");
   default:
     return string("UnknownObjectType");
   }
@@ -56,20 +41,6 @@ const string NativeObjectTypeToString(NativeObjectType type) {
 NativeObjectType NativeObjectTypeFromString(const string type) {
   if (type == "BatchHandlerType") {
     return BatchHandlerType;
-  } else if (type == "MapperType") {
-    return MapperType;
-  } else if (type == "ReducerType") {
-    return ReducerType;
-  } else if (type == "PartitionerType") {
-    return PartitionerType;
-  } else if (type == "CombinerType") {
-    return CombinerType;
-  } else if (type == "FolderType") {
-    return CombinerType;
-  } else if (type == "RecordReaderType") {
-    return RecordReaderType;
-  } else if (type == "RecordWriterType") {
-    return RecordWriterType;
   }
   return UnknownObjectType;
 }
@@ -96,7 +67,7 @@ HadoopException::HadoopException(const string & what) {
 #ifndef __CYGWIN__
   size = backtrace(array, 64);
   char ** traces = backtrace_symbols(array, size);
-  for (size_t i = 0; i< size;i++) {
+  for (size_t i = 0; i < size; i++) {
     _reason.append("\n\t");
     _reason.append(traces[i]);
   }
@@ -235,13 +206,6 @@ Counter * ProcessorBase::getCounter(const string & group, const string & name) {
   return NULL;
 }
 
-uint32_t Partitioner::getPartition(const char * key, uint32_t & keyLen, uint32_t numPartition) {
-  if (numPartition == 1) {
-    return 0;
-  }
-  return (Hash::BytesHash(key, keyLen) & 0x7fffffff) % numPartition;
-}
-
 ///////////////////////////////////////////////////////////
 
-}
+} // namespace NativeTask

+ 12 - 13
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/PartitionBucket.cc

@@ -16,19 +16,19 @@
  * limitations under the License.
  */
 
-#include "commons.h"
+#include "lib/commons.h"
 #include "util/Timer.h"
 #include "util/StringUtil.h"
-#include "NativeObjectFactory.h"
-#include "PartitionBucket.h"
-#include "Merge.h"
+#include "lib/NativeObjectFactory.h"
+#include "lib/PartitionBucket.h"
+#include "lib/Merge.h"
 #include "NativeTask.h"
-#include "WritableUtils.h"
+#include "util/WritableUtils.h"
 #include "util/DualPivotQuickSort.h"
-#include "Combiner.h"
-#include "TaskCounters.h"
-#include "MinHeap.h"
-#include "PartitionBucketIterator.h"
+#include "lib/Combiner.h"
+#include "lib/TaskCounters.h"
+#include "lib/MinHeap.h"
+#include "lib/PartitionBucketIterator.h"
 
 namespace NativeTask {
 
@@ -39,7 +39,8 @@ KVIterator * PartitionBucket::getIterator() {
   return new PartitionBucketIterator(this, _keyComparator);
 }
 
-void PartitionBucket::spill(IFileWriter * writer) throw (IOException, UnsupportException) {
+void PartitionBucket::spill(IFileWriter * writer)
+  throw(IOException, UnsupportException) {
   KVIterator * iterator = getIterator();
   if (NULL == iterator || NULL == writer) {
     return;
@@ -71,6 +72,4 @@ void PartitionBucket::sort(SortAlgorithm type) {
   _sorted = true;
 }
 
-}
-;
-// namespace NativeTask
+} // namespace NativeTask

+ 8 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/PartitionBucket.h

@@ -20,14 +20,14 @@
 #define PARTITION_BUCKET_H_
 
 #include "NativeTask.h"
-#include "MemoryPool.h"
-#include "MemoryBlock.h"
-#include "Timer.h"
-#include "Buffers.h"
-#include "MapOutputSpec.h"
-#include "IFile.h"
-#include "SpillInfo.h"
-#include "Combiner.h"
+#include "lib/MemoryPool.h"
+#include "lib/MemoryBlock.h"
+#include "util/Timer.h"
+#include "lib/Buffers.h"
+#include "lib/MapOutputSpec.h"
+#include "lib/IFile.h"
+#include "lib/SpillInfo.h"
+#include "lib/Combiner.h"
 
 namespace NativeTask {
 

+ 11 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/PartitionBucketIterator.cc

@@ -16,18 +16,20 @@
  * limitations under the License.
  */
 
-#include "commons.h"
+#include <algorithm>
+
+#include "lib/commons.h"
 #include "util/Timer.h"
 #include "util/StringUtil.h"
-#include "NativeObjectFactory.h"
-#include "PartitionBucketIterator.h"
-#include "Merge.h"
+#include "lib/NativeObjectFactory.h"
+#include "lib/PartitionBucketIterator.h"
+#include "lib/Merge.h"
 #include "NativeTask.h"
-#include "WritableUtils.h"
+#include "util/WritableUtils.h"
 #include "util/DualPivotQuickSort.h"
-#include "Combiner.h"
-#include "TaskCounters.h"
-#include "MinHeap.h"
+#include "lib/Combiner.h"
+#include "lib/TaskCounters.h"
+#include "lib/MinHeap.h"
 
 namespace NativeTask {
 
@@ -108,7 +110,5 @@ bool PartitionBucketIterator::next(Buffer & key, Buffer & value) {
   }
   return false;
 }
-}
-;
-// namespace NativeTask
+} // namespace NativeTask
 

+ 8 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/PartitionBucketIterator.h

@@ -20,14 +20,14 @@
 #define PARTITION_BUCKET_ITERATOR_H_
 
 #include "NativeTask.h"
-#include "MemoryPool.h"
-#include "Timer.h"
-#include "Buffers.h"
-#include "MapOutputSpec.h"
-#include "IFile.h"
-#include "SpillInfo.h"
-#include "Combiner.h"
-#include "PartitionBucket.h"
+#include "lib/MemoryPool.h"
+#include "util/Timer.h"
+#include "lib/Buffers.h"
+#include "lib/MapOutputSpec.h"
+#include "lib/IFile.h"
+#include "lib/SpillInfo.h"
+#include "lib/Combiner.h"
+#include "lib/PartitionBucket.h"
 
 namespace NativeTask {
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Path.cc

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-#include "Path.h"
+#include "lib/Path.h"
 
 namespace NativeTask {
 

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/SpillInfo.cc

@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 
-#include "commons.h"
-#include "Streams.h"
-#include "FileSystem.h"
-#include "Buffers.h"
-#include "SpillInfo.h"
+#include "lib/commons.h"
+#include "lib/Streams.h"
+#include "lib/FileSystem.h"
+#include "lib/Buffers.h"
+#include "lib/SpillInfo.h"
 
 namespace NativeTask {
 
@@ -69,5 +69,5 @@ void SingleSpillInfo::writeSpillInfo(const std::string & filepath) {
   delete fout;
 }
 
-}
+} // namespace NativeTask
 

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Streams.cc

@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-#include "commons.h"
-#include "Checksum.h"
-#include "Streams.h"
+#include "lib/commons.h"
+#include "util/Checksum.h"
+#include "lib/Streams.h"
 
 namespace NativeTask {
 

+ 1 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/TaskCounters.cc

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-#include "TaskCounters.h"
+#include "lib/TaskCounters.h"
 
 namespace NativeTask {
 
@@ -36,6 +36,5 @@ const char * TaskCounters::FILESYSTEM_COUNTER_GROUP = "FileSystemCounters";
 
 DEFINE_COUNTER(FILE_BYTES_READ)
 DEFINE_COUNTER(FILE_BYTES_WRITTEN)
-;
 
 } // namespace NativeTask

+ 0 - 50
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/TrackingCollector.h

@@ -1,50 +0,0 @@
-/**
- * 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.
- */
-
-#ifndef TRACKING_COLLECTOR_H
-#define TRACKING_COLLECTOR_H
-
-#include <stdint.h>
-#include <string>
-
-namespace NativeTask {
-
-class TrackingCollector : public Collector {
-protected:
-  Collector * _collector;
-  Counter * _counter;
-public:
-  TrackingCollector(Collector * collector, Counter * counter)
-      : _collector(collector), _counter(counter) {
-  }
-
-  virtual void collect(const void * key, uint32_t keyLen, const void * value, uint32_t valueLen) {
-    _counter->increase();
-    _collector->collect(key, keyLen, value, valueLen);
-  }
-
-  virtual void collect(const void * key, uint32_t keyLen, const void * value, uint32_t valueLen,
-      int32_t partition) {
-    _counter->increase();
-    _collector->collect(key, keyLen, value, valueLen, partition);
-  }
-};
-
-} //namespace NativeTask
-
-#endif //TRACKING_COLLECTOR_H

+ 4 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/commons.h

@@ -41,13 +41,12 @@
 #include <map>
 #include <algorithm>
 
-#include "primitives.h"
-#include "Log.h"
+#include "lib/primitives.h"
+#include "lib/Log.h"
 #include "NativeTask.h"
 
-#include "Constants.h"
+#include "lib/Constants.h"
 
-#include "Iterator.h"
-#include "TrackingCollector.h"
+#include "lib/Iterator.h"
 
 #endif /* COMMONS_H_ */

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/jniutils.cc

@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-#include "commons.h"
+#include "lib/commons.h"
 #include "util/StringUtil.h"
 #include "util/SyncUtils.h"
-#include "jniutils.h"
+#include "lib/jniutils.h"
 
 using namespace NativeTask;
 
@@ -53,7 +53,7 @@ JavaVM * JNU_GetJVM(void) {
         JavaVMOption options[noArgs];
         options[0].optionString = optHadoopClassPath;
 
-        //Create the VM
+        // Create the VM
         JavaVMInitArgs vm_args;
         vm_args.version = JNI_VERSION_1_6;
         vm_args.options = options;
@@ -108,4 +108,4 @@ std::string JNU_ByteArrayToString(JNIEnv * jenv, jbyteArray src) {
     return ret;
   }
   return std::string();
-}
+}

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Checksum.cc

@@ -17,7 +17,7 @@
  */
 
 #include <assert.h>
-#include "Checksum.h"
+#include "util/Checksum.h"
 
 namespace NativeTask {
 

+ 0 - 24
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Hash.cc

@@ -1,24 +0,0 @@
-/**
- * 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.
- */
-
-#include "Hash.h"
-
-namespace NativeTask {
-
-} // namespace NativeTask
-

+ 0 - 73
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Hash.h

@@ -1,73 +0,0 @@
-/**
- * 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.
- */
-
-#ifndef HASH_H_
-#define HASH_H_
-
-#include <stdint.h>
-#include <stdlib.h>
-
-// Hash function for a byte array.
-extern uint64_t CityHash64(const char *buf, size_t len);
-
-// Hash function for a byte array.  For convenience, a 64-bit seed is also
-// hashed into the result.
-extern uint64_t CityHash64WithSeed(const char *buf, size_t len, uint64_t seed);
-
-namespace NativeTask {
-
-class Hash {
-public:
-  /**
-   * Compatible with hadoop Text & BytesWritable hash
-   */
-  inline static int32_t BytesHash(const char * bytes, uint32_t length) {
-    int32_t hash = 1;
-    for (uint32_t i = 0; i < length; i++)
-      hash = (31 * hash) + (int32_t)bytes[i];
-    return hash;
-  }
-
-  /**
-   * Unsigned version of BytesHash
-   */
-  inline static uint32_t BytesHashU(const char * bytes, uint32_t length) {
-    uint32_t hash = 1;
-    for (uint32_t i = 0; i < length; i++)
-      hash = (31U * hash) + (uint32_t)bytes[i];
-    return hash;
-  }
-
-  /**
-   * City hash, faster for longer input
-   */
-  inline static uint64_t CityHash(const char * bytes, uint32_t length) {
-    return CityHash64(bytes, length);
-  }
-
-  /**
-   * City hash, faster for longer input
-   */
-  inline static uint64_t CityHashWithSeed(const char * bytes, uint32_t length, uint64_t seed) {
-    return CityHash64WithSeed(bytes, length, seed);
-  }
-};
-
-} // namespace NativeTask
-
-#endif /* HASH_H_ */

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Random.cc

@@ -17,8 +17,8 @@
  */
 
 #include <math.h>
-#include "commons.h"
-#include "Random.h"
+#include "lib/commons.h"
+#include "util/Random.h"
 
 namespace NativeTask {
 

+ 5 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/StringUtil.cc

@@ -17,8 +17,8 @@
  */
 
 #include <stdarg.h>
-#include "commons.h"
-#include "StringUtil.h"
+#include "lib/commons.h"
+#include "util/StringUtil.h"
 
 namespace NativeTask {
 
@@ -201,8 +201,9 @@ bool StringUtil::StartsWith(const string & str, const string & prefix) {
 }
 
 bool StringUtil::EndsWith(const string & str, const string & suffix) {
-  if ((suffix.length() > str.length())
-      || (memcmp(str.data() + str.length() - suffix.length(), suffix.data(), suffix.length()) != 0)) {
+  if ((suffix.length() > str.length()) ||
+      (memcmp(str.data() + str.length() - suffix.length(),
+              suffix.data(), suffix.length()) != 0)) {
     return false;
   }
   return true;

+ 4 - 110
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/SyncUtils.cc

@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-#include "commons.h"
-#include "jniutils.h"
-#include "StringUtil.h"
-#include "SyncUtils.h"
+#include "lib/commons.h"
+#include "lib/jniutils.h"
+#include "util/StringUtil.h"
+#include "util/SyncUtils.h"
 
 namespace NativeTask {
 
@@ -52,110 +52,4 @@ void Lock::unlock() {
   PthreadCall("unlock", pthread_mutex_unlock(&_mutex));
 }
 
-#ifdef __MACH__
-SpinLock::SpinLock() : _spin(0) {
-}
-
-SpinLock::~SpinLock() {
-
-}
-
-void SpinLock::lock() {
-  OSSpinLockLock(&_spin);
-}
-
-void SpinLock::unlock() {
-  OSSpinLockUnlock(&_spin);
-}
-#else
-SpinLock::SpinLock() {
-  PthreadCall("init mutex", pthread_spin_init(&_spin, 0));
-}
-
-SpinLock::~SpinLock() {
-  PthreadCall("destroy mutex", pthread_spin_destroy(&_spin));
-}
-
-void SpinLock::lock() {
-  PthreadCall("lock", pthread_spin_lock(&_spin));
-}
-
-void SpinLock::unlock() {
-  PthreadCall("unlock", pthread_spin_unlock(&_spin));
-}
-#endif
-
-Condition::Condition(Lock* mu)
-    : _lock(mu) {
-  PthreadCall("init cv", pthread_cond_init(&_condition, NULL));
-}
-
-Condition::~Condition() {
-  PthreadCall("destroy cv", pthread_cond_destroy(&_condition));
-}
-
-void Condition::wait() {
-  PthreadCall("wait", pthread_cond_wait(&_condition, &_lock->_mutex));
-}
-
-void Condition::signal() {
-  PthreadCall("signal", pthread_cond_signal(&_condition));
-}
-
-void Condition::signalAll() {
-  PthreadCall("broadcast", pthread_cond_broadcast(&_condition));
-}
-
-void * Thread::ThreadRunner(void * pthis) {
-  try {
-    ((Thread*)pthis)->run();
-  } catch (std::exception & e) {
-    LOG("err!!!! %s", e.what());
-  }
-  return NULL;
-}
-
-Thread::Thread()
-    : _thread((pthread_t)0), // safe for linux & macos
-    _runable(NULL) {
-}
-
-Thread::Thread(Runnable * runnable)
-    : _thread((pthread_t)0), _runable(runnable) {
-}
-
-void Thread::setTask(const Runnable & runnable) {
-  _runable = const_cast<Runnable*>(&runnable);
-}
-
-Thread::~Thread() {
-
-}
-
-void Thread::start() {
-  PthreadCall("pthread_create", pthread_create(&_thread, NULL, ThreadRunner, this));
-}
-
-void Thread::join() {
-  PthreadCall("pthread_join", pthread_join(_thread, NULL));
-}
-
-void Thread::stop() {
-  PthreadCall("pthread_cancel", pthread_cancel(_thread));
-}
-
-void Thread::run() {
-  if (_runable != NULL) {
-    _runable->run();
-  }
-}
-
-void Thread::EnableJNI() {
-  JNU_AttachCurrentThread();
-}
-
-void Thread::ReleaseJNI() {
-  JNU_DetachCurrentThread();
-}
-
 } // namespace NativeTask

+ 0 - 228
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/SyncUtils.h

@@ -47,38 +47,6 @@ private:
   void operator=(const Lock&);
 };
 
-class SpinLock {
-public:
-  SpinLock();
-  ~SpinLock();
-
-  void lock();
-  void unlock();
-
-private:
-#ifdef __MACH__
-  OSSpinLock _spin;
-#else
-  pthread_spinlock_t _spin;
-#endif
-
-  // No copying
-  SpinLock(const Lock&);
-  void operator=(const Lock&);
-};
-
-class Condition {
-public:
-  explicit Condition(Lock* mu);
-  ~Condition();
-  void wait();
-  void signal();
-  void signalAll();
-private:
-  pthread_cond_t _condition;
-  Lock* _lock;
-};
-
 template<typename LockT>
 class ScopeLock {
 public:
@@ -97,202 +65,6 @@ private:
   void operator=(const ScopeLock&);
 };
 
-class Runnable {
-public:
-  virtual ~Runnable() {
-  }
-  virtual void run() = 0;
-};
-
-class Thread : public Runnable {
-protected:
-  pthread_t _thread;
-  Runnable * _runable;
-public:
-  Thread();
-  Thread(Runnable * runnable);
-  virtual ~Thread();
-
-  void setTask(const Runnable & runnable);
-  void start();
-  void join();
-  void stop();
-  virtual void run();
-
-  /**
-   * Enable JNI for current thread
-   */
-  static void EnableJNI();
-  /**
-   * Release JNI when thread is at end if current
-   * thread called EnableJNI before
-   */
-  static void ReleaseJNI();
-private:
-  static void * ThreadRunner(void * pthis);
-};
-
-// Sure <tr1/functional> is better
-template<typename Subject, typename Method>
-class FunctionRunner : public Runnable {
-protected:
-  Subject & _subject;
-  Method _method;
-public:
-  FunctionRunner(Subject & subject, Method method)
-      : _subject(subject), _method(method) {
-  }
-
-  virtual void run() {
-    (_subject.*_method)();
-  }
-};
-
-template<typename Subject, typename Method, typename Arg>
-class FunctionRunner1 : public Runnable {
-protected:
-  Subject & _subject;
-  Method _method;
-  Arg _arg;
-public:
-  FunctionRunner1(Subject & subject, Method method, Arg arg)
-      : _subject(subject), _method(method), _arg(arg) {
-  }
-
-  virtual void run() {
-    (_subject.*_method)(_arg);
-  }
-};
-
-template<typename Subject, typename Method, typename Arg1, typename Arg2>
-class FunctionRunner2 : public Runnable {
-protected:
-  Subject & _subject;
-  Method _method;
-  Arg1 _arg1;
-  Arg2 _arg2;
-public:
-  FunctionRunner2(Subject & subject, Method method, Arg1 arg1, Arg2 arg2)
-      : _subject(subject), _method(method), _arg1(arg1), _arg2(arg2) {
-  }
-
-  virtual void run() {
-    (_subject.*_method)(_arg1, _arg2);
-  }
-};
-
-template<typename Subject, typename Method>
-inline FunctionRunner<Subject, Method> * BindNew(Subject & subject, Method method) {
-  return new FunctionRunner<Subject, Method>(subject, method);
-}
-
-template<typename Subject, typename Method, typename Arg>
-inline FunctionRunner1<Subject, Method, Arg> * BindNew(Subject & subject, Method method, Arg arg) {
-  return new FunctionRunner1<Subject, Method, Arg>(subject, method, arg);
-}
-
-template<typename Subject, typename Method, typename Arg1, typename Arg2>
-inline FunctionRunner2<Subject, Method, Arg1, Arg2> * BindNew(Subject & subject, Method method,
-    Arg1 arg1, Arg2 arg2) {
-  return new FunctionRunner2<Subject, Method, Arg1, Arg2>(subject, method, arg1, arg2);
-}
-
-class ConcurrentIndex {
-private:
-  size_t _index;
-  size_t _end;
-  SpinLock _lock;
-public:
-  ConcurrentIndex(size_t count)
-      : _index(0), _end(count) {
-  }
-
-  ConcurrentIndex(size_t start, size_t end)
-      : _index(start), _end(end) {
-  }
-
-  size_t count() {
-    return _end;
-  }
-
-  ssize_t next() {
-    ScopeLock<SpinLock> autoLock(_lock);
-    if (_index == _end) {
-      return -1;
-    } else {
-      ssize_t ret = _index;
-      _index++;
-      return ret;
-    }
-  }
-};
-
-template<typename Subject, typename Method, typename RangeType>
-class ParallelForWorker : public Runnable {
-protected:
-  ConcurrentIndex * _index;
-  Subject * _subject;
-  Method _method;
-public:
-  ParallelForWorker()
-      : _index(NULL), _subject(NULL) {
-  }
-
-  ParallelForWorker(ConcurrentIndex * index, Subject * subject, Method method)
-      : _index(index), _subject(subject), _method(method) {
-  }
-
-  void reset(ConcurrentIndex * index, Subject * subject, Method method) {
-    _index = index;
-    _subject = subject;
-    _method = method;
-  }
-
-  virtual void run() {
-    ssize_t i;
-    while ((i = _index->next()) >= 0) {
-      (_subject->*_method)(i);
-    }
-  }
-};
-
-template<typename Subject, typename Method, typename RangeType>
-void ParallelFor(Subject & subject, Method method, RangeType begin, RangeType end,
-    size_t thread_num) {
-  RangeType count = end - begin;
-  if (thread_num <= 1 || count <= 1) {
-    for (RangeType i = begin; i < end; i++) {
-      (subject.*method)(i);
-    }
-  } else if (thread_num == 2) {
-    ConcurrentIndex index = ConcurrentIndex(begin, end);
-    ParallelForWorker<Subject, Method, RangeType> workers[2];
-    Thread sideThread;
-    workers[0].reset(&index, &subject, method);
-    workers[1].reset(&index, &subject, method);
-    sideThread.setTask(workers[0]);
-    sideThread.start();
-    workers[1].run();
-    sideThread.join();
-  } else {
-    ConcurrentIndex index = ConcurrentIndex(begin, end);
-    ParallelForWorker<Subject, Method, RangeType> * workers = new ParallelForWorker<Subject, Method,
-        RangeType> [thread_num];
-    Thread * threads = new Thread[thread_num - 1];
-    for (size_t i = 0; i < thread_num - 1; i++) {
-      workers[i].reset(&index, &subject, method);
-      threads[i].setTask(workers[i]);
-      threads[i].start();
-    }
-    workers[thread_num - 1].reset(&index, &subject, method);
-    workers[thread_num - 1].run();
-    for (size_t i = 0; i < thread_num - 1; i++) {
-      threads[i].join();
-    }
-    delete[] threads;
-    delete[] workers;
-  }
-}
 
 } // namespace NativeTask
 

+ 3 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/Timer.cc

@@ -17,9 +17,9 @@
  */
 
 #include <time.h>
-#include "commons.h"
-#include "StringUtil.h"
-#include "Timer.h"
+#include "lib/commons.h"
+#include "util/StringUtil.h"
+#include "util/Timer.h"
 
 namespace NativeTask {
 
@@ -51,7 +51,6 @@ Timer::Timer() {
 }
 
 Timer::~Timer() {
-
 }
 
 uint64_t Timer::last() {

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/WritableUtils.cc

@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-#include "commons.h"
-#include "StringUtil.h"
-#include "WritableUtils.h"
+#include "lib/commons.h"
+#include "util/StringUtil.h"
+#include "util/WritableUtils.h"
 
 namespace NativeTask {
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/WritableUtils.h

@@ -21,7 +21,7 @@
 
 #include <stdint.h>
 #include <string>
-#include "Streams.h"
+#include "lib/Streams.h"
 #include "NativeTask.h"
 
 namespace NativeTask {

+ 5 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestCommand.cc

@@ -16,13 +16,13 @@
  * limitations under the License.
  */
 
-#include "commons.h"
-#include "BufferStream.h"
-#include "Buffers.h"
+#include "lib/commons.h"
+#include "lib/BufferStream.h"
+#include "lib/Buffers.h"
 #include "test_commons.h"
 #include "NativeTask.h"
 
-using namespace NativeTask;
+namespace NativeTask {
 
 TEST(Command, equals) {
   Command cmd1(100, "hello command");
@@ -36,3 +36,4 @@ TEST(Command, equals) {
   ASSERT_EQ(helloCommand, cmd1.description());
 }
 
+} // namespace NativeTask

+ 6 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestCompressions.cc

@@ -18,11 +18,11 @@
 
 #include "lz4.h"
 #include "config.h"
-#include "commons.h"
-#include "Path.h"
-#include "BufferStream.h"
-#include "FileSystem.h"
-#include "Compressions.h"
+#include "lib/commons.h"
+#include "lib/Path.h"
+#include "lib/BufferStream.h"
+#include "lib/FileSystem.h"
+#include "lib/Compressions.h"
 #include "test_commons.h"
 
 #if defined HADOOP_SNAPPY_LIBRARY
@@ -144,7 +144,7 @@ TEST(Perf, CompressionUtil) {
 }
 
 class CompressResult {
-public:
+ public:
   uint64_t uncompressedSize;
   uint64_t compressedSize;
   uint64_t compressTime;
@@ -193,10 +193,8 @@ void MeasureSingleFileLz4(const string & path, CompressResult & total, size_t bl
     result.compressTime += endTime - startTime;
     startTime = t.now();
     for (int i = 0; i < times; i++) {
-//      memset(dest, 0, currentblocksize+8);
       int osize = LZ4_uncompress(outputBuffer, dest, currentblocksize);
       ASSERT_EQ(currentblocksize, osize);
-//      printf("%016llx blocksize: %lu\n", bswap64(*(uint64_t*)(dest+currentblocksize)), currentblocksize);
     }
     endTime = t.now();
     result.uncompressTime += endTime - startTime;

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestConfig.cc

@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-#include "commons.h"
-#include "BufferStream.h"
-#include "Buffers.h"
+#include "lib/commons.h"
+#include "lib/BufferStream.h"
+#include "lib/Buffers.h"
 #include "test_commons.h"
 
 float absoute(float v) {

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestCounter.cc

@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-#include "commons.h"
-#include "NativeObjectFactory.h"
-#include "BufferStream.h"
-#include "Buffers.h"
+#include "lib/commons.h"
+#include "lib/NativeObjectFactory.h"
+#include "lib/BufferStream.h"
+#include "lib/Buffers.h"
 #include "test_commons.h"
 
 TEST(Counter, Counter) {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestFileSystem.cc

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-#include "FileSystem.h"
+#include "lib/FileSystem.h"
 #include "test_commons.h"
 
 TEST(FileSystem, RawFileSystem) {

+ 7 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestIFile.cc

@@ -17,11 +17,11 @@
  */
 
 #include <algorithm>
-#include "commons.h"
+#include "lib/commons.h"
 #include "config.h"
-#include "BufferStream.h"
-#include "FileSystem.h"
-#include "IFile.h"
+#include "lib/BufferStream.h"
+#include "lib/FileSystem.h"
+#include "lib/IFile.h"
 #include "test_commons.h"
 
 SingleSpillInfo * writeIFile(int partition, vector<pair<string, string> > & kvs,
@@ -166,8 +166,8 @@ TEST(Perf, IFile) {
   delete[] buff;
 }
 
-// The Glibc has a bug in the file tell api, it will overwrite the file data 
-// unexpected. 
+// The Glibc has a bug in the file tell api, it will overwrite the file data
+// unexpected.
 // Please check https://rhn.redhat.com/errata/RHBA-2013-0279.html
 // This case is to check whether the bug exists.
 // If it exists, it means you need to upgrade the glibc.
@@ -189,12 +189,11 @@ TEST(IFile, TestGlibCBug) {
   uint32_t length = 0;
   reader->nextPartition();
   uint32_t index = 0;
-  while(NULL != (key = reader->nextKey(length))) {
+  while (NULL != (key = reader->nextKey(length))) {
     int32_t realKey = (int32_t)bswap(*(uint32_t *)(key));
     ASSERT_LT(index, 5);
     ASSERT_EQ(expect[index], realKey);
     index++;
   }
   delete reader;
-
 }

+ 14 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestMain.cc

@@ -23,10 +23,10 @@
 #endif
 
 #include <stdexcept>
-#include "commons.h"
-#include "Buffers.h"
-#include "FileSystem.h"
-#include "NativeObjectFactory.h"
+#include "lib/commons.h"
+#include "lib/Buffers.h"
+#include "lib/FileSystem.h"
+#include "lib/NativeObjectFactory.h"
 #include "test_commons.h"
 
 extern "C" {
@@ -52,11 +52,11 @@ void handler(int sig) {
 }
 }
 
-using namespace NativeTask;
-
 typedef char * CString;
 
-int main(int argc, char ** argv) {
+namespace NativeTask {
+
+int DoMain(int argc, char** argv) {
   signal(SIGSEGV, handler);
   CString * newArgv = new CString[argc + 1];
   memcpy(newArgv, argv, argc * sizeof(CString));
@@ -107,3 +107,10 @@ int main(int argc, char ** argv) {
     return 1;
   }
 }
+
+} // namespace NativeTask
+
+
+int main(int argc, char ** argv) {
+  return NativeTask::DoMain(argc, argv);
+}

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestPrimitives.cc

@@ -166,7 +166,7 @@ inline char * memchrbrf2(char * p, char ch, size_t len) {
 
 // not safe in MACOSX, segment fault, should be safe on Linux with out mmap
 inline int memchr_sse(const char *s, int c, int len) {
-  //len : edx; c: esi; s:rdi
+  // len : edx; c: esi; s:rdi
   int index = 0;
 
 #ifdef __X64

+ 8 - 25
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestSort.cc

@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-#include "commons.h"
-#include "Streams.h"
-#include "Buffers.h"
-#include "DualPivotQuickSort.h"
+#include "lib/commons.h"
+#include "lib/Streams.h"
+#include "lib/Buffers.h"
+#include "util/DualPivotQuickSort.h"
 #include "test_commons.h"
 
 string gBuffer;
@@ -82,7 +82,7 @@ static int compare_offset(const void * plh, const void * prh) {
  * dualpivot sort compare function
  */
 class CompareOffset {
-public:
+ public:
   int64_t operator()(uint32_t lhs, uint32_t rhs) {
 
     KVBuffer * lhb = (KVBuffer*)get_position(lhs);
@@ -101,7 +101,7 @@ public:
  * quicksort compare function
  */
 class OffsetLessThan {
-public:
+ public:
   bool operator()(uint32_t lhs, uint32_t rhs) {
     KVBuffer * lhb = (KVBuffer*)get_position(lhs);
     KVBuffer * rhb = (KVBuffer*)get_position(rhs);
@@ -132,7 +132,7 @@ static int compare_offset2(const void * plh, const void * prh) {
  * dualpivot sort compare function
  */
 class CompareOffset2 {
-public:
+ public:
   int64_t operator()(uint32_t lhs, uint32_t rhs) {
 
     KVBuffer * lhb = (KVBuffer*)get_position(lhs);
@@ -151,7 +151,7 @@ public:
  * quicksort compare function
  */
 class OffsetLessThan2 {
-public:
+ public:
   bool operator()(uint32_t lhs, uint32_t rhs) {
 
     KVBuffer * lhb = (KVBuffer*)get_position(lhs);
@@ -163,23 +163,6 @@ public:
   }
 };
 
-/*
-void makeInput(string & dest, vector<uint32_t> & offsets, uint64_t length) {
-  TeraGen tera = TeraGen(length / 100, 1, 0);
-  dest.reserve(length + 1024);
-  string k, v;
-  while (tera.next(k, v)) {
-    offsets.push_back(dest.length());
-    uint32_t tempLen = k.length();
-    dest.append((const char *)&tempLen, 4);
-    dest.append(k.data(), k.length());
-    tempLen = v.length();
-    dest.append((const char *)&tempLen, 4);
-    dest.append(v.data(), v.length());
-  }
-}
-*/
-
 void makeInputWord(string & dest, vector<uint32_t> & offsets, uint64_t length) {
   Random r;
   dest.reserve(length + 1024);

Một số tệp đã không được hiển thị bởi vì quá nhiều tập tin thay đổi trong này khác