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

This commit is contained in:
Todd Lipcon 2014-08-27 12:23:03 -07:00
parent 6d39367f6a
commit fad4524c85
23 changed files with 59 additions and 56 deletions

View File

@ -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)

View File

@ -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}
)

View File

@ -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; \

View File

@ -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;

View File

@ -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;

View File

@ -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);

View File

@ -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;
}

View File

@ -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);
}

View File

@ -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;

View File

@ -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() {

View File

@ -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,

View File

@ -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;

View File

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

View File

@ -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;
}

View File

@ -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,

View File

@ -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;

View File

@ -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);

View File

@ -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);
}

View File

@ -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;
}

View File

@ -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);

View File

@ -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());

View File

@ -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());