MAPREDUCE-5977. Fix or suppress native-task gcc warnings. Contributed by Manu Zhang.
This commit is contained in:
parent
6d39367f6a
commit
fad4524c85
|
@ -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)
|
||||
|
||||
|
|
|
@ -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}
|
||||
)
|
||||
|
||||
|
|
|
@ -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; \
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -30,6 +30,9 @@
|
|||
#include <memory.h>
|
||||
#include <fcntl.h>
|
||||
|
||||
#define __STDC_FORMAT_MACROS
|
||||
#include <inttypes.h>
|
||||
|
||||
#include <limits>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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());
|
||||
|
|
Loading…
Reference in New Issue