Forráskód Böngészése

MAPREDUCE-5977. Fix or suppress native-task gcc warnings. Contributed by Manu Zhang.

Todd Lipcon 10 éve
szülő
commit
fad4524c85
23 módosított fájl, 59 hozzáadás és 56 törlés
  1. 2 0
      hadoop-mapreduce-project/CHANGES.MAPREDUCE-2841.txt
  2. 6 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt
  3. 0 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest/include/gtest/gtest.h
  4. 0 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/NativeTask.h
  5. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/codec/BlockCodec.cc
  6. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/handler/BatchHandler.h
  7. 4 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/handler/CombineHandler.cc
  8. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Buffers.h
  9. 5 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/IFile.cc
  10. 7 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MapOutputCollector.cc
  11. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/MapOutputCollector.h
  12. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/Merge.cc
  13. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/PartitionBucket.h
  14. 3 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/commons.h
  15. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/util/StringUtil.cc
  16. 3 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestCompressions.cc
  17. 4 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/TestIFile.cc
  18. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/lib/TestByteBuffer.cc
  19. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/lib/TestFixSizeContainer.cc
  20. 0 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/lib/TestIterator.cc
  21. 2 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/lib/TestMemBlockIterator.cc
  22. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/lib/TestMemoryBlock.cc
  23. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/util/TestHash.cc

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

@@ -15,3 +15,5 @@ MAPREDUCE-5978. native-task: remove test case for not supported codec Bzip2Codec
 MAPREDUCE-6006. native-task: add native tests to maven and fix bug in pom.xml (Binglin Chang via todd)
 MAPREDUCE-6026. native-task: fix logging (Manu Zhang via todd)
 MAPREDUCE-6035. native-task: sources/test-sources jar distribution (Manu Zhang via todd)
+MAPREDUCE-5977. Fix or suppress native-task gcc warnings (Manu Zhang via todd)
+

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

@@ -90,7 +90,7 @@ SET(CMAKE_FIND_LIBRARY_SUFFIXES STORED_CMAKE_FIND_LIBRARY_SUFFIXES)
 
 # primitive configs
 set(PRFLAGS "-DSIMPLE_MEMCPY")
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${PRFLAGS} -Wall")
+set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${PRFLAGS} -fno-strict-aliasing -Wall -Wno-sign-compare")
 set(CMAKE_LD_FLAGS "${CMAKE_LD_FLAGS} -no-undefined -version-info 0:1:0
     -L${_JAVA_HOME}/jre/lib/amd64/server -ljvm")
 set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${CMAKE_C_FLAGS} -g -O2 -DNDEBUG -fPIC")
@@ -150,7 +150,6 @@ CONFIGURE_FILE(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
 
 include_directories(
     ${GENERATED_JAVAH}
-    ${D}
     ${D}/src
     ${D}/src/util
     ${D}/src/lib
@@ -160,6 +159,8 @@ include_directories(
     ${JNI_INCLUDE_DIRS}
     ${SNAPPY_INCLUDE_DIR}
 )
+# add gtest as system library to suppress gcc warnings
+include_directories(SYSTEM ${D}/gtest/include)
 
 
 SET(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
@@ -220,8 +221,9 @@ add_dual_library(nativetask
 
 target_link_libraries(nativetask ${NT_DEPEND_LIBRARY})
 
+add_library(gtest ${D}/gtest/gtest-all.cc)
+set_target_properties(gtest PROPERTIES COMPILE_FLAGS "-w")
 add_executable(nttest
-    ${D}/gtest/gtest-all.cc
     ${D}/test/lib/TestByteArray.cc
     ${D}/test/lib/TestByteBuffer.cc
     ${D}/test/lib/TestComparatorForDualPivotQuickSort.cc
@@ -260,6 +262,7 @@ ENDIF()
 
 target_link_libraries(nttest
      nativetask_static
+     gtest
      ${NT_DEPEND_LIBRARY}
 )
 

+ 0 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest/gtest.h → hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest/include/gtest/gtest.h


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

@@ -622,7 +622,6 @@ typedef void (*ANY_FUNC_PTR)();
 #define DEFINE_NATIVE_LIBRARY(Library) \
   static std::map<std::string, NativeTask::ObjectCreatorFunc> Library##ClassMap__; \
   extern "C" void * Library##GetFunctionGetter(const std::string & name) { \
-      void * ret = NULL; \
       std::map<std::string, NativeTask::ObjectCreatorFunc>::iterator itr = Library##ClassMap__.find(name); \
       if (itr != Library##ClassMap__.end()) { \
         return (void *)(itr->second); \
@@ -630,7 +629,6 @@ typedef void (*ANY_FUNC_PTR)();
       return NULL; \
     } \
   extern "C" NativeTask::ObjectCreatorFunc Library##GetObjectCreator(const std::string & name) { \
-    NativeObject * ret = NULL; \
     std::map<std::string, NativeTask::ObjectCreatorFunc>::iterator itr = Library##ClassMap__.find(name); \
     if (itr != Library##ClassMap__.end()) { \
       return itr->second; \

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

@@ -23,7 +23,7 @@
 namespace NativeTask {
 
 BlockCompressStream::BlockCompressStream(OutputStream * stream, uint32_t bufferSizeHint)
-    : CompressStream(stream), _compressedBytesWritten(0), _tempBufferSize(0), _tempBuffer(NULL) {
+    : CompressStream(stream), _tempBuffer(NULL), _tempBufferSize(0), _compressedBytesWritten(0) {
   _hint = bufferSizeHint;
   _blockMax = bufferSizeHint / 2 * 3;
 }
@@ -68,7 +68,7 @@ uint64_t BlockCompressStream::compressedBytesWritten() {
 //////////////////////////////////////////////////////////////
 
 BlockDecompressStream::BlockDecompressStream(InputStream * stream, uint32_t bufferSizeHint)
-    : DecompressStream(stream), _tempBufferSize(0), _tempBuffer(NULL) {
+    : DecompressStream(stream), _tempBuffer(NULL), _tempBufferSize(0) {
   _hint = bufferSizeHint;
   _blockMax = bufferSizeHint / 2 * 3;
   _tempDecompressBuffer = NULL;

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/handler/BatchHandler.h

@@ -103,10 +103,10 @@ protected:
    */
   inline void output(const char * buff, uint32_t length) {
     while (length > 0) {
-      if (length > _out.remain()) {
+      uint32_t remain = _out.remain();
+      if (length > remain) {
         flushOutput();
       }
-      uint32_t remain = _out.remain();
       uint32_t cp = length < remain ? length : remain;
       simple_memcpy(_out.current(), buff, cp);
       buff += cp;

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

@@ -24,9 +24,9 @@ const int LENGTH_OF_REFILL_STRING = 6;
 const Command CombineHandler::COMBINE(4, "Combine");
 
 CombineHandler::CombineHandler()
-    : _combineContext(NULL), _kvIterator(NULL), _writer(NULL), _config(NULL), _kvCached(false),
-      _kType(UnknownType), _vType(UnknownType), _combineInputRecordCount(0), _combineInputBytes(0),
-      _combineOutputRecordCount(0),_combineOutputBytes(0){
+    : _combineContext(NULL), _kvIterator(NULL), _writer(NULL), _kType(UnknownType),
+        _vType(UnknownType), _config(NULL), _kvCached(false), _combineInputRecordCount(0),
+        _combineInputBytes(0), _combineOutputRecordCount(0), _combineOutputBytes(0) {
 }
 
 CombineHandler::~CombineHandler() {
@@ -95,7 +95,6 @@ uint32_t CombineHandler::feedDataToJavaInWritableSerialization() {
  * KV: key or value
  */
 void CombineHandler::outputKeyOrValue(SerializeInfo & KV, KeyValueType type) {
-  uint32_t length = 0;
   switch (type) {
   case TextType:
     output(KV.varBytes, KV.outerLength - KV.buffer.length());
@@ -159,7 +158,6 @@ uint32_t CombineHandler::feedDataToJava(SerializationFramework serializationType
 void CombineHandler::handleInput(ByteBuffer & in) {
   char * buff = in.current();
   uint32_t length = in.remain();
-  const char * end = buff + length;
   uint32_t remain = length;
   char * pos = buff;
   if (_asideBuffer.remain() > 0) {
@@ -183,7 +181,7 @@ void CombineHandler::handleInput(ByteBuffer & in) {
     THROW_EXCEPTION(IOException, "k/v meta information incomplete");
   }
 
-  int kvLength = kvBuffer->lengthConvertEndium();
+  uint32_t kvLength = kvBuffer->lengthConvertEndium();
 
   if (kvLength > remain) {
     _asideBytes.resize(kvLength);

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

@@ -429,8 +429,8 @@ public:
 
   void rewind(int newPos, int newLimit) {
     this->_position = newPos;
-    if (newLimit > this->_capacity) {
-      THROW_EXCEPTION(IOException, "length larger than input buffer capacity");
+    if (newLimit < 0 || newLimit > this->_capacity) {
+      THROW_EXCEPTION(IOException, "length smaller than zero or larger than input buffer capacity");
     }
     this->_limit = newLimit;
   }

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

@@ -27,9 +27,9 @@ namespace NativeTask {
 ///////////////////////////////////////////////////////////
 
 IFileReader::IFileReader(InputStream * stream, SingleSpillInfo * spill, bool deleteInputStream)
-    : _deleteSourceStream(deleteInputStream), _stream(stream), _source(NULL),
-        _checksumType(spill->checkSumType), _kType(spill->keyType), _vType(spill->valueType),
-        _codec(spill->codec), _segmentIndex(-1), _spillInfo(spill), _valuePos(NULL), _valueLen(0) {
+    :  _stream(stream), _source(NULL), _checksumType(spill->checkSumType), _kType(spill->keyType),
+        _vType(spill->valueType), _codec(spill->codec), _segmentIndex(-1), _spillInfo(spill),
+        _valuePos(NULL), _valueLen(0), _deleteSourceStream(deleteInputStream) {
   _source = new ChecksumInputStream(_stream, _checksumType);
   _source->setLimit(0);
   _reader.init(128 * 1024, _source, _codec);
@@ -97,9 +97,8 @@ IFileWriter * IFileWriter::create(const std::string & filepath, const MapOutputS
 
 IFileWriter::IFileWriter(OutputStream * stream, ChecksumType checksumType, KeyValueType ktype,
     KeyValueType vtype, const string & codec, Counter * counter, bool deleteTargetStream)
-    : _deleteTargetStream(deleteTargetStream), _stream(stream), _dest(NULL),
-        _checksumType(checksumType), _kType(ktype), _vType(vtype), _codec(codec),
-        _recordCounter(counter) {
+    : _stream(stream), _dest(NULL), _checksumType(checksumType), _kType(ktype), _vType(vtype),
+        _codec(codec), _recordCounter(counter), _deleteTargetStream(deleteTargetStream) {
   _dest = new ChecksumOutputStream(_stream, _checksumType);
   _appendBuffer.init(128 * 1024, _dest, _codec);
 }

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

@@ -76,9 +76,9 @@ void CombineRunnerWrapper::combine(CombineContext type, KVIterator * kvIterator,
 /////////////////////////////////////////////////////////////////
 
 MapOutputCollector::MapOutputCollector(uint32_t numberPartitions, SpillOutputService * spillService)
-    : _config(NULL), _buckets(NULL), _keyComparator(NULL), _defaultBlockSize(0),
-        _combineRunner(NULL), _spilledRecords(NULL), _spillOutput(spillService), _pool(NULL),
-        _numPartitions(numberPartitions) {
+    : _config(NULL), _numPartitions(numberPartitions), _buckets(NULL), _keyComparator(NULL),
+        _combineRunner(NULL), _spilledRecords(NULL), _spillOutput(spillService),
+        _defaultBlockSize(0),  _pool(NULL) {
   _pool = new MemoryPool();
 }
 
@@ -186,7 +186,7 @@ KVBuffer * MapOutputCollector::allocateKVBuffer(uint32_t partitionId, uint32_t k
       delete spillpath;
     }
 
-    dest = dest = partition->allocateKVBuffer(kvlength);
+    dest = partition->allocateKVBuffer(kvlength);
     if (NULL == dest) {
       // io.sort.mb too small, cann't proceed
       // should not get here, cause get_buffer_to_put can throw OOM exception
@@ -294,7 +294,7 @@ void MapOutputCollector::middleSpill(const std::string & spillOutput,
     uint64_t spillTime = timer.now() - timer.last() - metrics.sortTime;
 
     LOG(
-        "[MapOutputCollector::mid_spill] Sort and spill: {spilled file path: %s, id: %d, collect: %llu ms, sort: %llu ms, spill: %llu ms, records: %llu, uncompressed total bytes: %llu, compressed total bytes: %llu}",
+        "[MapOutputCollector::mid_spill] Sort and spill: {spilled file path: %s, id: %d, collect: %"PRIu64" ms, sort: %"PRIu64" ms, spill: %"PRIu64" ms, records: %"PRIu64", uncompressed total bytes: %"PRIu64", compressed total bytes: %"PRIu64"}",
         info->path.c_str(), _spillInfos.getSpillCount(), collecttime / M, metrics.sortTime  / M, spillTime  / M,
         metrics.recordCount, info->getEndPosition(), info->getRealEndPosition());
 
@@ -339,7 +339,7 @@ void MapOutputCollector::finalSpill(const std::string & filepath,
 
   SortMetrics metrics;
   sortPartitions(_spec.sortOrder, _spec.sortAlgorithm, NULL, metrics);
-  LOG("[MapOutputCollector::mid_spill] Sort final in memory kvs: {sort: %llu ms, records: %llu}",
+  LOG("[MapOutputCollector::mid_spill] Sort final in memory kvs: {sort: %"PRIu64" ms, records: %"PRIu64"}",
       metrics.sortTime / M, metrics.recordCount);
 
   merger->addMergeEntry(new MemoryMergeEntry(_buckets, _numPartitions));
@@ -347,7 +347,7 @@ void MapOutputCollector::finalSpill(const std::string & filepath,
   Timer timer;
   merger->merge();
   LOG(
-      "[MapOutputCollector::final_merge_and_spill]  Merge and Spill:{spilled file id: %d, merge and spill time: %llu ms}",
+      "[MapOutputCollector::final_merge_and_spill]  Merge and Spill:{spilled file id: %d, merge and spill time: %"PRIu64" ms}",
       _spillInfos.getSpillCount(), (timer.now() - timer.last()) / M);
 
   delete merger;

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

@@ -55,8 +55,8 @@ private:
 
 public:
   CombineRunnerWrapper(Config * config, SpillOutputService * service)
-      : _spillOutput(service), _config(config), _isJavaCombiner(false), _combineRunner(NULL),
-          _combinerInited(false) {
+      : _config(config), _combineRunner(NULL), _isJavaCombiner(false),
+          _combinerInited(false), _spillOutput(service) {
   }
 
   ~CombineRunnerWrapper() {

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

@@ -162,7 +162,7 @@ void Merger::merge() {
   _writer->getStatistics(output_size, real_output_size);
 
   if (total_record != 0) {
-    LOG("[Merge] Merged segment#: %lu, record#: %llu, avg record size: %llu, uncompressed total bytes: %llu, compressed total bytes: %llu, time: %llu ms",
+    LOG("[Merge] Merged segment#: %lu, record#: %"PRIu64", avg record size: %"PRIu64", uncompressed total bytes: %"PRIu64", compressed total bytes: %"PRIu64", time: %"PRIu64" ms",
         _entries.size(),
         total_record,
         output_size / (total_record),
@@ -170,7 +170,7 @@ void Merger::merge() {
         real_output_size,
         interval / M);
   } else {
-    LOG("[Merge] Merged segments#, %lu, uncompressed total bytes: %llu, compressed total bytes: %llu, time: %llu ms",
+    LOG("[Merge] Merged segments#, %lu, uncompressed total bytes: %"PRIu64", compressed total bytes: %"PRIu64", time: %"PRIu64" ms",
         _entries.size(),
         output_size,
         real_output_size,

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

@@ -50,8 +50,8 @@ private:
 public:
   PartitionBucket(MemoryPool * pool, uint32_t partition, ComparatorPtr comparator,
       ICombineRunner * combineRunner, uint32_t blockSize)
-      : _pool(pool), _partition(partition), _keyComparator(comparator),
-          _combineRunner(combineRunner), _blockSize(blockSize), _sorted(false) {
+      : _pool(pool), _partition(partition), _blockSize(blockSize),
+          _keyComparator(comparator), _combineRunner(combineRunner),  _sorted(false) {
     if (NULL == _pool || NULL == comparator) {
       THROW_EXCEPTION_EX(IOException, "pool is NULL, or comparator is not set");
     }
@@ -102,11 +102,11 @@ public:
     }
     _sorted = false;
     MemoryBlock * memBlock = NULL;
-    uint32_t memBockSize = _memBlocks.size();
-    if (memBockSize > 0) {
-      memBlock = _memBlocks[memBockSize - 1];
+    uint32_t memBlockSize = _memBlocks.size();
+    if (memBlockSize > 0) {
+      memBlock = _memBlocks[memBlockSize - 1];
     }
-    if (NULL != memBockSize && memBlock->remainSpace() >= kvLength) {
+    if (NULL != memBlock && memBlock->remainSpace() >= kvLength) {
       return memBlock->allocateKVBuffer(kvLength);
     } else {
       uint32_t min = kvLength;

+ 3 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/src/lib/commons.h

@@ -30,6 +30,9 @@
 #include <memory.h>
 #include <fcntl.h>
 
+#define __STDC_FORMAT_MACROS
+#include <inttypes.h>
+
 #include <limits>
 #include <string>
 #include <vector>

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

@@ -36,19 +36,19 @@ string StringUtil::ToString(uint32_t v) {
 
 string StringUtil::ToString(int64_t v) {
   char tmp[32];
-  snprintf(tmp, 32, "%lld", (long long int)v);
+  snprintf(tmp, 32, "%"PRId64, v);
   return tmp;
 }
 
 string StringUtil::ToString(int64_t v, char pad, int64_t len) {
   char tmp[32];
-  snprintf(tmp, 32, "%%%c%lldlld", pad, len);
+  snprintf(tmp, 32, "%%%c%"PRId64""PRId64, pad, len);
   return Format(tmp, v);
 }
 
 string StringUtil::ToString(uint64_t v) {
   char tmp[32];
-  snprintf(tmp, 32, "%llu", (long long unsigned int)v);
+  snprintf(tmp, 32, "%"PRIu64, v);
   return tmp;
 }
 

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

@@ -181,13 +181,11 @@ void MeasureSingleFileLz4(const string & path, CompressResult & total, size_t bl
   char * dest = new char[blockSize + 8];
   CompressResult result;
   Timer t;
-  int compressedSize;
   for (size_t start = 0; start < data.length(); start += blockSize) {
     size_t currentblocksize = std::min(data.length() - start, blockSize);
     uint64_t startTime = t.now();
     for (int i = 0; i < times; i++) {
       int osize = LZ4_compress((char*)data.data() + start, outputBuffer, currentblocksize);
-      compressedSize = osize;
       result.compressedSize += osize;
       result.uncompressedSize += currentblocksize;
     }
@@ -197,6 +195,7 @@ void MeasureSingleFileLz4(const string & path, CompressResult & total, size_t bl
     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();
@@ -239,7 +238,7 @@ void MeasureSingleFileSnappy(const string & path, CompressResult & total, size_t
   char * dest = new char[blockSize];
   CompressResult result;
   Timer t;
-  int compressedSize;
+  int compressedSize = -1;
   for (size_t start = 0; start < data.length(); start += blockSize) {
     size_t currentblocksize = std::min(data.length() - start, blockSize);
     uint64_t startTime = t.now();
@@ -272,7 +271,7 @@ TEST(Perf, RawCompressionSnappy) {
   vector<FileEntry> inputfiles;
   FileSystem::getLocal().list(inputdir, inputfiles);
   CompressResult total;
-  printf("Block size: %lldK\n", blockSize / 1024);
+  printf("Block size: %"PRId64"K\n", blockSize / 1024);
   for (size_t i = 0; i < inputfiles.size(); i++) {
     if (!inputfiles[i].isDirectory) {
       MeasureSingleFileSnappy((inputdir + "/" + inputfiles[i].name).c_str(), total, blockSize,

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

@@ -119,13 +119,17 @@ void TestIFileWriteRead2(vector<pair<string, string> > & kvs, char * buff, size_
   InputBuffer inputBuffer = InputBuffer(buff, outputBuffer.tell());
   IFileReader * ir = new IFileReader(&inputBuffer, info);
   timer.reset();
+  int sum = 0;
   while (ir->nextPartition()) {
     const char * key, *value;
     uint32_t keyLen, valueLen;
     while (NULL != (key = ir->nextKey(keyLen))) {
       value = ir->value(valueLen);
+      sum += value[0];
     }
   }
+  // use the result so that value() calls don't get optimized out
+  ASSERT_NE(0xdeadbeef, sum);
   LOG("%s",
       timer.getSpeedM2(" Read data", info->getEndPosition(), info->getRealEndPosition()).c_str());
   delete ir;

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

@@ -35,7 +35,7 @@ TEST(ByteBuffer, read) {
   ASSERT_EQ(buff, byteBuffer.current());
   ASSERT_EQ(0, byteBuffer.remain());
 
-  int newPos = byteBuffer.advance(3);
+  byteBuffer.advance(3);
   ASSERT_EQ(3, byteBuffer.current() - byteBuffer.base());
 
   byteBuffer.rewind(10, 20);

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

@@ -43,7 +43,7 @@ TEST(FixSizeContainer, test) {
 
   container->fill(toBeFilled.c_str(), toBeFilled.length());
 
-  for (int i = 0; i < container->position(); i++) {
+  for (uint32_t i = 0; i < container->position(); i++) {
     char * c = container->base() + i;
     ASSERT_EQ(toBeFilled[i], *c);
   }

+ 0 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/lib/TestIterator.cc

@@ -88,8 +88,6 @@ void TestKeyGroupIterator() {
     int * keyPtr = (int *)key;
     const char * value = NULL;
     while (NULL != (value = groupIterator->nextValue(length))) {
-      int * valuePtr = (int *)value;
-
       if (actualKeyCount.find(*keyPtr) == actualKeyCount.end()) {
         actualKeyCount[*keyPtr] = 0;
       }

+ 2 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/lib/TestMemBlockIterator.cc

@@ -27,13 +27,12 @@ namespace NativeTask {
 
 TEST(MemoryBlockIterator, test) {
   const uint32_t BUFFER_LENGTH = 100;
-  const uint32_t BLOCK_ID = 3;
   char * bytes = new char[BUFFER_LENGTH];
   MemoryBlock block(bytes, BUFFER_LENGTH);
 
   const uint32_t KV_SIZE = 60;
-  KVBuffer * kv1 = block.allocateKVBuffer(KV_SIZE);
-  KVBuffer * kv2 = block.allocateKVBuffer(KV_SIZE);
+  block.allocateKVBuffer(KV_SIZE);
+  block.allocateKVBuffer(KV_SIZE);
 
   MemBlockIterator iter(&block);
 
@@ -50,7 +49,6 @@ class MemoryBlockFactory {
 public:
   static MemoryBlock * create(std::vector<int> & keys) {
     const uint32_t BUFFER_LENGTH = 1000;
-    const uint32_t BLOCK_ID = 3;
     char * bytes = new char[BUFFER_LENGTH];
     MemoryBlock * block1 = new MemoryBlock(bytes, BUFFER_LENGTH);
 

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/test/lib/TestMemoryBlock.cc

@@ -44,6 +44,7 @@ TEST(MemoryBlock, test) {
 
   ASSERT_EQ(2, block.getKVCount());
   ASSERT_EQ(kv1, block.getKVBuffer(0));
+  ASSERT_EQ(kv2, block.getKVBuffer(1));
 
   ASSERT_EQ(BUFFER_LENGTH - 2 * KV_SIZE, block.remainSpace());
   ASSERT_EQ(false, block.sorted());

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

@@ -24,7 +24,7 @@ static uint64_t test_length(int64_t len, size_t size, size_t loopTime) {
   TestConfig.setInt(GenerateLen, len);
   Generate(data, size, "bytes");
   Timer t;
-  uint64_t ret;
+  uint64_t ret = 0;
   for (size_t m = 0; m < loopTime; m++) {
     for (size_t i = 0; i < data.size(); i++) {
       ret += Hash::BytesHash(data[i].c_str(), data[i].length());