MAPREDUCE-2841. Merge remote-tracking branch 'apache/MR-2841' into trunk
This commit is contained in:
commit
8a3a327cd3
33
LICENSE.txt
33
LICENSE.txt
|
@ -288,3 +288,36 @@ lz4_encoder.h,lz4hc.h,lz4hc.c,lz4hc_encoder.h},
|
|||
- LZ4 homepage : http://fastcompression.blogspot.com/p/lz4.html
|
||||
- LZ4 source repository : http://code.google.com/p/lz4/
|
||||
*/
|
||||
|
||||
|
||||
For hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest
|
||||
---------------------------------------------------------------------
|
||||
Copyright 2008, Google Inc.
|
||||
All rights reserved.
|
||||
|
||||
Redistribution and use in source and binary forms, with or without
|
||||
modification, are permitted provided that the following conditions are
|
||||
met:
|
||||
|
||||
* Redistributions of source code must retain the above copyright
|
||||
notice, this list of conditions and the following disclaimer.
|
||||
* Redistributions in binary form must reproduce the above
|
||||
copyright notice, this list of conditions and the following disclaimer
|
||||
in the documentation and/or other materials provided with the
|
||||
distribution.
|
||||
* Neither the name of Google Inc. nor the names of its
|
||||
contributors may be used to endorse or promote products derived from
|
||||
this software without specific prior written permission.
|
||||
|
||||
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
|
||||
THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
|
||||
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
|
||||
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
|
||||
|
|
|
@ -135,6 +135,13 @@
|
|||
<include>*-sources.jar</include>
|
||||
</includes>
|
||||
</fileSet>
|
||||
<fileSet>
|
||||
<directory>hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/target</directory>
|
||||
<outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>
|
||||
<includes>
|
||||
<include>*-sources.jar</include>
|
||||
</includes>
|
||||
</fileSet>
|
||||
<fileSet>
|
||||
<directory>hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/target</directory>
|
||||
<outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>
|
||||
|
|
|
@ -0,0 +1,26 @@
|
|||
Changes for Hadoop Native Map Output Collector
|
||||
------------------------------------------------
|
||||
|
||||
MAPREDUCE-5985. native-task: Fix build on macosx. Contributed by Binglin Chang
|
||||
MAPREDUCE-5994. Simplify ByteUtils and fix failing test. (todd)
|
||||
MAPREDUCE-5996. native-task: Rename system tests into standard directory layout (todd)
|
||||
MAPREDUCE-5997. native-task: Use DirectBufferPool from Hadoop Common (todd)
|
||||
MAPREDUCE-6000. native-task: Simplify ByteBufferDataReader/Writer (todd)
|
||||
MAPREDUCE-5991. native-task should not run unit tests if native profile is not enabled. (Binglin Chang)
|
||||
MAPREDUCE-5995. native-task: Revert changes to Text internals (todd)
|
||||
MAPREDUCE-6005. native-task: Fix some valgrind errors (Binglin Chang)
|
||||
MAPREDUCE-5984. native-task: Reuse lz4 sources in hadoop-common (Binglin Chang)
|
||||
MAPREDUCE-5976. native-task: should not fail to build if snappy is missing (Manu Zhang)
|
||||
MAPREDUCE-5978. native-task: remove test case for not supported codec Bzip2Codec and DefaultCodec (Manu Zhang)
|
||||
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)
|
||||
MAPREDUCE-6054. native-task: Speed up tests (todd)
|
||||
MAPREDUCE-6058. native-task: KVTest and LargeKVTest should check mr job is sucessful (Binglin Chang)
|
||||
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)
|
||||
MAPREDUCE-6074. native-task: fix release audit, javadoc, javac warnings (todd)
|
||||
MAPREDUCE-6077. native-task: Remove CustomModule examples in nativetask (seanzhong)
|
|
@ -0,0 +1,216 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
Licensed 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. See accompanying LICENSE file.
|
||||
-->
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
|
||||
http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<parent>
|
||||
<artifactId>hadoop-mapreduce-client</artifactId>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<version>3.0.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-client-nativetask</artifactId>
|
||||
<version>3.0.0-SNAPSHOT</version>
|
||||
<name>hadoop-mapreduce-client-nativetask</name>
|
||||
|
||||
<properties>
|
||||
<!-- Needed for generating FindBugs warnings using parent pom -->
|
||||
<mr.basedir>${project.parent.basedir}/../</mr.basedir>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-client-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.avro</groupId>
|
||||
<artifactId>avro</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-all</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-client-common</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-surefire-plugin</artifactId>
|
||||
<configuration>
|
||||
<properties>
|
||||
<property>
|
||||
<name>listener</name>
|
||||
<value>org.apache.hadoop.test.TimedOutTestsListener</value>
|
||||
</property>
|
||||
</properties>
|
||||
<includes>
|
||||
<include>**/*Test.java</include>
|
||||
<include>**/Test*.java</include>
|
||||
</includes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes>
|
||||
<exclude>src/main/native/testData/*</exclude>
|
||||
<!-- gtest has a compatible license -->
|
||||
<exclude>src/main/native/gtest/**/*</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
<profiles>
|
||||
<profile>
|
||||
<id>native</id>
|
||||
<activation>
|
||||
<activeByDefault>false</activeByDefault>
|
||||
</activation>
|
||||
<properties>
|
||||
<snappy.prefix></snappy.prefix>
|
||||
<snappy.lib></snappy.lib>
|
||||
<snappy.include></snappy.include>
|
||||
<require.snappy>false</require.snappy>
|
||||
</properties>
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-enforcer-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>enforce-os</id>
|
||||
<goals>
|
||||
<goal>enforce</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<rules>
|
||||
<requireOS>
|
||||
<family>mac</family>
|
||||
<family>unix</family>
|
||||
<message>native build only supported on Mac or
|
||||
Unix</message>
|
||||
</requireOS>
|
||||
</rules>
|
||||
<fail>true</fail>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.codehaus.mojo</groupId>
|
||||
<artifactId>native-maven-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<phase>compile</phase>
|
||||
<goals>
|
||||
<goal>javah</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<javahPath>${env.JAVA_HOME}/bin/javah</javahPath>
|
||||
<javahClassNames>
|
||||
<javaClassName>org.apache.hadoop.mapred.nativetask.NativeBatchProcessor</javaClassName>
|
||||
<javaClassName>org.apache.hadoop.mapred.nativetask.NativeRuntime</javaClassName>
|
||||
</javahClassNames>
|
||||
<javahOutputDirectory>${project.build.directory}/native/javah</javahOutputDirectory>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-antrun-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>make</id>
|
||||
<phase>compile</phase>
|
||||
<goals>
|
||||
<goal>run</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<target>
|
||||
<copy file="${basedir}/../../../hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4.h"
|
||||
todir="${project.build.directory}/native/" />
|
||||
<copy file="${basedir}/../../../hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4_encoder.h"
|
||||
todir="${project.build.directory}/native/" />
|
||||
<copy file="${basedir}/../../../hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4.c"
|
||||
todir="${project.build.directory}/native/" />
|
||||
<copy todir="${project.build.directory}/native/test/testData"
|
||||
overwrite="true">
|
||||
<fileset dir="${basedir}/src/main/native/testData" />
|
||||
</copy>
|
||||
<exec executable="cmake" dir="${project.build.directory}/native"
|
||||
failonerror="true">
|
||||
<arg line="${basedir}/src/ -DGENERATED_JAVAH=${project.build.directory}/native/javah -DJVM_ARCH_DATA_MODEL=${sun.arch.data.model} -DREQUIRE_SNAPPY=${require.snappy} -DCUSTOM_SNAPPY_PREFIX=${snappy.prefix} -DCUSTOM_SNAPPY_LIB=${snappy.lib} -DCUSTOM_SNAPPY_INCLUDE=${snappy.include}" />
|
||||
</exec>
|
||||
<exec executable="make" dir="${project.build.directory}/native" failonerror="true">
|
||||
<arg line="VERBOSE=1" />
|
||||
</exec>
|
||||
<!-- The second make is a workaround for HADOOP-9215. It can be
|
||||
removed when version 2.6 of cmake is no longer supported . -->
|
||||
<exec executable="make" dir="${project.build.directory}/native" failonerror="true"></exec>
|
||||
</target>
|
||||
</configuration>
|
||||
</execution>
|
||||
<execution>
|
||||
<id>native_tests</id>
|
||||
<phase>test</phase>
|
||||
<goals><goal>run</goal></goals>
|
||||
<configuration>
|
||||
<target>
|
||||
<exec executable="sh" failonerror="true" dir="${project.build.directory}/native/test">
|
||||
<arg value="-c"/>
|
||||
<arg value="[ x$SKIPTESTS = xtrue ] || sh test.sh"/>
|
||||
<env key="SKIPTESTS" value="${skipTests}"/>
|
||||
</exec>
|
||||
</target>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</profile>
|
||||
</profiles>
|
||||
</project>
|
|
@ -0,0 +1,276 @@
|
|||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
|
||||
|
||||
# Default to release builds
|
||||
set(CMAKE_BUILD_TYPE, Release)
|
||||
|
||||
include(JNIFlags.cmake NO_POLICY_SCOPE)
|
||||
|
||||
# Compile a library with both shared and static variants
|
||||
function(add_dual_library LIBNAME)
|
||||
add_library(${LIBNAME} SHARED ${ARGN})
|
||||
add_library(${LIBNAME}_static STATIC ${ARGN})
|
||||
set_target_properties(${LIBNAME}_static PROPERTIES OUTPUT_NAME ${LIBNAME})
|
||||
endfunction(add_dual_library)
|
||||
|
||||
# Link both a static and a dynamic target against some libraries
|
||||
function(target_link_dual_libraries LIBNAME)
|
||||
target_link_libraries(${LIBNAME} ${ARGN})
|
||||
target_link_libraries(${LIBNAME}_static ${ARGN})
|
||||
endfunction(target_link_dual_libraries)
|
||||
|
||||
function(output_directory TGT DIR)
|
||||
SET_TARGET_PROPERTIES(${TGT} PROPERTIES
|
||||
RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
|
||||
SET_TARGET_PROPERTIES(${TGT} PROPERTIES
|
||||
ARCHIVE_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
|
||||
SET_TARGET_PROPERTIES(${TGT} PROPERTIES
|
||||
LIBRARY_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
|
||||
endfunction(output_directory TGT DIR)
|
||||
|
||||
function(dual_output_directory TGT DIR)
|
||||
output_directory(${TGT} "${DIR}")
|
||||
output_directory(${TGT}_static "${DIR}")
|
||||
endfunction(dual_output_directory TGT DIR)
|
||||
|
||||
#
|
||||
# This macro alters the behavior of find_package and find_library.
|
||||
# It does this by setting the CMAKE_FIND_LIBRARY_SUFFIXES global variable.
|
||||
# You should save that variable before calling this function and restore it
|
||||
# after you have accomplished your goal.
|
||||
#
|
||||
# The behavior is altered in two ways:
|
||||
# 1. We always find shared libraries, never static;
|
||||
# 2. We find shared libraries with the given version number.
|
||||
#
|
||||
# On Windows this function is a no-op. Windows does not encode
|
||||
# version number information information into library path names.
|
||||
#
|
||||
macro(set_find_shared_library_version LVERS)
|
||||
IF(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
|
||||
# Mac OS uses .dylib
|
||||
SET(CMAKE_FIND_LIBRARY_SUFFIXES ".${LVERS}.dylib")
|
||||
ELSEIF(${CMAKE_SYSTEM_NAME} MATCHES "FreeBSD")
|
||||
# FreeBSD has always .so installed.
|
||||
SET(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
|
||||
ELSEIF(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
|
||||
# Windows doesn't support finding shared libraries by version.
|
||||
ELSE()
|
||||
# Most UNIX variants use .so
|
||||
SET(CMAKE_FIND_LIBRARY_SUFFIXES ".so.${LVERS}")
|
||||
ENDIF()
|
||||
endmacro(set_find_shared_library_version LVERS)
|
||||
|
||||
if (NOT GENERATED_JAVAH)
|
||||
#Must identify where the generated headers have been placed
|
||||
MESSAGE(FATAL_ERROR "You must set the cmake variable GENERATED_JAVAH")
|
||||
endif (NOT GENERATED_JAVAH)
|
||||
find_package(JNI REQUIRED)
|
||||
|
||||
SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES CMAKE_FIND_LIBRARY_SUFFIXES)
|
||||
set_find_shared_library_version("1")
|
||||
SET(CMAKE_FIND_LIBRARY_SUFFIXES STORED_CMAKE_FIND_LIBRARY_SUFFIXES)
|
||||
|
||||
# primitive configs
|
||||
set(PRFLAGS "-DSIMPLE_MEMCPY")
|
||||
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")
|
||||
set(D main/native/)
|
||||
|
||||
SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES CMAKE_FIND_LIBRARY_SUFFIXES)
|
||||
set_find_shared_library_version("1")
|
||||
SET(CMAKE_FIND_LIBRARY_SUFFIXES STORED_CMAKE_FIND_LIBRARY_SUFFIXES)
|
||||
|
||||
INCLUDE(CheckFunctionExists)
|
||||
INCLUDE(CheckCSourceCompiles)
|
||||
#INCLUDE(CheckLibraryExists)
|
||||
INCLUDE(CheckIncludeFiles)
|
||||
#CHECK_FUNCTION_EXISTS(sync_file_range HAVE_SYNC_FILE_RANGE)
|
||||
#CHECK_FUNCTION_EXISTS(posix_fadvise HAVE_POSIX_FADVISE)
|
||||
#CHECK_LIBRARY_EXISTS(dl dlopen "" NEED_LINK_DL)
|
||||
CHECK_INCLUDE_FILES(fcntl.h HAVE_FCNTL_H)
|
||||
CHECK_INCLUDE_FILES(malloc.h HAVE_MALLOC_H)
|
||||
CHECK_INCLUDE_FILES(mach/mach.h HAVE_MACH_MACH_H)
|
||||
CHECK_INCLUDE_FILES(memory.h HAVE_MEMORY_H)
|
||||
CHECK_INCLUDE_FILES(stddef.h HAVE_STDDEF_H)
|
||||
CHECK_INCLUDE_FILES(stdint.h HAVE_STDINT_H)
|
||||
CHECK_INCLUDE_FILES(stdlib.h HAVE_STDLIB_H)
|
||||
CHECK_INCLUDE_FILES(string.h HAVE_STRING_H)
|
||||
CHECK_INCLUDE_FILES(unistd.h HAVE_UNITSTD_H)
|
||||
CHECK_FUNCTION_EXISTS(clock_gettime HAVE_CLOCK_GETTIME)
|
||||
CHECK_FUNCTION_EXISTS(localtime_r HAVE_LOCALTIME_R)
|
||||
CHECK_FUNCTION_EXISTS(memset HAVE_MEMSET)
|
||||
CHECK_FUNCTION_EXISTS(strchr HAVE_STRCHR)
|
||||
CHECK_FUNCTION_EXISTS(strtoul HAVE_STRTOUL)
|
||||
|
||||
SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES CMAKE_FIND_LIBRARY_SUFFIXES)
|
||||
set_find_shared_library_version("1")
|
||||
find_library(SNAPPY_LIBRARY
|
||||
NAMES snappy
|
||||
PATHS ${CUSTOM_SNAPPY_PREFIX} ${CUSTOM_SNAPPY_PREFIX}/lib
|
||||
${CUSTOM_SNAPPY_PREFIX}/lib64 ${CUSTOM_SNAPPY_LIB})
|
||||
SET(CMAKE_FIND_LIBRARY_SUFFIXES STORED_CMAKE_FIND_LIBRARY_SUFFIXES)
|
||||
find_path(SNAPPY_INCLUDE_DIR
|
||||
NAMES snappy.h
|
||||
PATHS ${CUSTOM_SNAPPY_PREFIX} ${CUSTOM_SNAPPY_PREFIX}/include
|
||||
${CUSTOM_SNAPPY_INCLUDE})
|
||||
if (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
|
||||
GET_FILENAME_COMPONENT(HADOOP_SNAPPY_LIBRARY ${SNAPPY_LIBRARY} NAME)
|
||||
set(SNAPPY_SOURCE_FILES
|
||||
"${D}/src/codec/SnappyCodec.cc")
|
||||
else (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
|
||||
set(SNAPPY_LIBRARY "")
|
||||
set(SNAPPY_INCLUDE_DIR "")
|
||||
set(SNAPPY_SOURCE_FILES "")
|
||||
IF(REQUIRE_SNAPPY)
|
||||
MESSAGE(FATAL_ERROR "Required snappy library could not be found. SNAPPY_LIBRARY=${SNAPPY_LIBRARY}, SNAPPY_INCLUDE_DIR=${SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_INCLUDE_DIR=${CUSTOM_SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_PREFIX=${CUSTOM_SNAPPY_PREFIX}, CUSTOM_SNAPPY_INCLUDE=${CUSTOM_SNAPPY_INCLUDE}")
|
||||
ENDIF(REQUIRE_SNAPPY)
|
||||
endif (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
|
||||
|
||||
CONFIGURE_FILE(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
|
||||
|
||||
include_directories(
|
||||
${GENERATED_JAVAH}
|
||||
${D}/src
|
||||
${D}/src/util
|
||||
${D}/src/lib
|
||||
${D}/test
|
||||
${CMAKE_CURRENT_SOURCE_DIR}
|
||||
${CMAKE_BINARY_DIR}
|
||||
${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)
|
||||
|
||||
if (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
|
||||
# macosx does not have -lrt
|
||||
set(NT_DEPEND_LIBRARY dl pthread z ${SNAPPY_LIBRARY} ${JAVA_JVM_LIBRARY})
|
||||
set(SYSTEM_MAC TRUE)
|
||||
else (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
|
||||
set(NT_DEPEND_LIBRARY dl rt pthread z ${SNAPPY_LIBRARY} ${JAVA_JVM_LIBRARY})
|
||||
set(SYSTEM_MAC FALSE)
|
||||
endif (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
|
||||
|
||||
configure_file(main/native/test.sh test/test.sh)
|
||||
|
||||
add_dual_library(nativetask
|
||||
${CMAKE_BINARY_DIR}/lz4.c
|
||||
${D}/src/codec/BlockCodec.cc
|
||||
${D}/src/codec/GzipCodec.cc
|
||||
${D}/src/codec/Lz4Codec.cc
|
||||
${SNAPPY_SOURCE_FILES}
|
||||
${D}/src/handler/BatchHandler.cc
|
||||
${D}/src/handler/MCollectorOutputHandler.cc
|
||||
${D}/src/handler/AbstractMapHandler.cc
|
||||
${D}/src/handler/CombineHandler.cc
|
||||
${D}/src/lib/Buffers.cc
|
||||
${D}/src/lib/BufferStream.cc
|
||||
${D}/src/lib/Compressions.cc
|
||||
${D}/src/lib/PartitionBucket.cc
|
||||
${D}/src/lib/PartitionBucketIterator.cc
|
||||
${D}/src/lib/FileSystem.cc
|
||||
${D}/src/lib/IFile.cc
|
||||
${D}/src/lib/jniutils.cc
|
||||
${D}/src/lib/Log.cc
|
||||
${D}/src/lib/MapOutputCollector.cc
|
||||
${D}/src/lib/MapOutputSpec.cc
|
||||
${D}/src/lib/MemoryBlock.cc
|
||||
${D}/src/lib/Merge.cc
|
||||
${D}/src/lib/NativeLibrary.cc
|
||||
${D}/src/lib/Iterator.cc
|
||||
${D}/src/lib/NativeObjectFactory.cc
|
||||
${D}/src/lib/NativeRuntimeJniImpl.cc
|
||||
${D}/src/lib/NativeTask.cc
|
||||
${D}/src/lib/SpillInfo.cc
|
||||
${D}/src/lib/Path.cc
|
||||
${D}/src/lib/Streams.cc
|
||||
${D}/src/lib/TaskCounters.cc
|
||||
${D}/src/util/Checksum.cc
|
||||
${D}/src/util/Random.cc
|
||||
${D}/src/util/StringUtil.cc
|
||||
${D}/src/util/SyncUtils.cc
|
||||
${D}/src/util/Timer.cc
|
||||
${D}/src/util/WritableUtils.cc
|
||||
)
|
||||
|
||||
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}/test/lib/TestByteArray.cc
|
||||
${D}/test/lib/TestByteBuffer.cc
|
||||
${D}/test/lib/TestComparatorForDualPivotQuickSort.cc
|
||||
${D}/test/lib/TestComparatorForStdSort.cc
|
||||
${D}/test/lib/TestFixSizeContainer.cc
|
||||
${D}/test/lib/TestMemoryPool.cc
|
||||
${D}/test/lib/TestIterator.cc
|
||||
${D}/test/lib/TestKVBuffer.cc
|
||||
${D}/test/lib/TestMemBlockIterator.cc
|
||||
${D}/test/lib/TestMemoryBlock.cc
|
||||
${D}/test/lib/TestPartitionBucket.cc
|
||||
${D}/test/lib/TestReadBuffer.cc
|
||||
${D}/test/lib/TestReadWriteBuffer.cc
|
||||
${D}/test/util/TestChecksum.cc
|
||||
${D}/test/util/TestStringUtil.cc
|
||||
${D}/test/util/TestWritableUtils.cc
|
||||
${D}/test/TestCommand.cc
|
||||
${D}/test/TestConfig.cc
|
||||
${D}/test/TestCounter.cc
|
||||
${D}/test/TestCompressions.cc
|
||||
${D}/test/TestFileSystem.cc
|
||||
${D}/test/TestIFile.cc
|
||||
${D}/test/TestPrimitives.cc
|
||||
${D}/test/TestSort.cc
|
||||
${D}/test/TestMain.cc
|
||||
${D}/test/test_commons.cc)
|
||||
|
||||
|
||||
IF (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
|
||||
# macos clang with libc++ does not have tr1/tuple, just tuple
|
||||
SET_TARGET_PROPERTIES(nttest PROPERTIES COMPILE_FLAGS "-DGTEST_USE_OWN_TR1_TUPLE=1")
|
||||
ENDIF()
|
||||
|
||||
target_link_libraries(nttest
|
||||
nativetask_static
|
||||
gtest
|
||||
${NT_DEPEND_LIBRARY}
|
||||
)
|
||||
|
||||
IF (${CMAKE_SYSTEM_NAME} MATCHES "Linux")
|
||||
#
|
||||
# By embedding '$ORIGIN' into the RPATH of libnativetask.so,
|
||||
# dlopen will look in the directory containing libnativetask.so.
|
||||
# However, $ORIGIN is not supported by all operating systems.
|
||||
#
|
||||
SET_TARGET_PROPERTIES(nativetask
|
||||
PROPERTIES INSTALL_RPATH "\$ORIGIN/")
|
||||
ENDIF()
|
||||
|
||||
SET(LIBNATIVETASK_VERSION "1.0.0")
|
||||
SET_TARGET_PROPERTIES(nativetask PROPERTIES SOVERSION ${LIBNATIVETASK_VERSION})
|
||||
dual_output_directory(nativetask target/usr/local/lib)
|
||||
output_directory(nttest test)
|
|
@ -0,0 +1,118 @@
|
|||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
|
||||
|
||||
# If JVM_ARCH_DATA_MODEL is 32, compile all binaries as 32-bit.
|
||||
# This variable is set by maven.
|
||||
if (JVM_ARCH_DATA_MODEL EQUAL 32)
|
||||
# Force 32-bit code generation on amd64/x86_64, ppc64, sparc64
|
||||
if (CMAKE_COMPILER_IS_GNUCC AND CMAKE_SYSTEM_PROCESSOR MATCHES ".*64")
|
||||
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -m32")
|
||||
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -m32")
|
||||
set(CMAKE_LD_FLAGS "${CMAKE_LD_FLAGS} -m32")
|
||||
endif ()
|
||||
if (CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
|
||||
# Set CMAKE_SYSTEM_PROCESSOR to ensure that find_package(JNI) will use
|
||||
# the 32-bit version of libjvm.so.
|
||||
set(CMAKE_SYSTEM_PROCESSOR "i686")
|
||||
endif ()
|
||||
endif (JVM_ARCH_DATA_MODEL EQUAL 32)
|
||||
|
||||
# Determine float ABI of JVM on ARM Linux
|
||||
if (CMAKE_SYSTEM_PROCESSOR MATCHES "^arm" AND CMAKE_SYSTEM_NAME STREQUAL "Linux")
|
||||
find_program(READELF readelf)
|
||||
if (READELF MATCHES "NOTFOUND")
|
||||
message(WARNING "readelf not found; JVM float ABI detection disabled")
|
||||
else (READELF MATCHES "NOTFOUND")
|
||||
execute_process(
|
||||
COMMAND ${READELF} -A ${JAVA_JVM_LIBRARY}
|
||||
OUTPUT_VARIABLE JVM_ELF_ARCH
|
||||
ERROR_QUIET)
|
||||
if (NOT JVM_ELF_ARCH MATCHES "Tag_ABI_VFP_args: VFP registers")
|
||||
message("Soft-float JVM detected")
|
||||
|
||||
# Test compilation with -mfloat-abi=softfp using an arbitrary libc function
|
||||
# (typically fails with "fatal error: bits/predefs.h: No such file or directory"
|
||||
# if soft-float dev libraries are not installed)
|
||||
include(CMakePushCheckState)
|
||||
cmake_push_check_state()
|
||||
set(CMAKE_REQUIRED_FLAGS "${CMAKE_REQUIRED_FLAGS} -mfloat-abi=softfp")
|
||||
include(CheckSymbolExists)
|
||||
check_symbol_exists(exit stdlib.h SOFTFP_AVAILABLE)
|
||||
if (NOT SOFTFP_AVAILABLE)
|
||||
message(FATAL_ERROR "Soft-float dev libraries required (e.g. 'apt-get install libc6-dev-armel' on Debian/Ubuntu)")
|
||||
endif (NOT SOFTFP_AVAILABLE)
|
||||
cmake_pop_check_state()
|
||||
|
||||
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mfloat-abi=softfp")
|
||||
endif ()
|
||||
endif (READELF MATCHES "NOTFOUND")
|
||||
endif (CMAKE_SYSTEM_PROCESSOR MATCHES "^arm" AND CMAKE_SYSTEM_NAME STREQUAL "Linux")
|
||||
|
||||
IF("${CMAKE_SYSTEM}" MATCHES "Linux")
|
||||
#
|
||||
# Locate JNI_INCLUDE_DIRS and JNI_LIBRARIES.
|
||||
# Since we were invoked from Maven, we know that the JAVA_HOME environment
|
||||
# variable is valid. So we ignore system paths here and just use JAVA_HOME.
|
||||
#
|
||||
FILE(TO_CMAKE_PATH "$ENV{JAVA_HOME}" _JAVA_HOME)
|
||||
IF(CMAKE_SYSTEM_PROCESSOR MATCHES "^i.86$")
|
||||
SET(_java_libarch "i386")
|
||||
ELSEIF (CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
|
||||
SET(_java_libarch "amd64")
|
||||
ELSEIF (CMAKE_SYSTEM_PROCESSOR MATCHES "^arm")
|
||||
SET(_java_libarch "arm")
|
||||
ELSE()
|
||||
SET(_java_libarch ${CMAKE_SYSTEM_PROCESSOR})
|
||||
ENDIF()
|
||||
SET(_JDK_DIRS "${_JAVA_HOME}/jre/lib/${_java_libarch}/*"
|
||||
"${_JAVA_HOME}/jre/lib/${_java_libarch}"
|
||||
"${_JAVA_HOME}/jre/lib/*"
|
||||
"${_JAVA_HOME}/jre/lib"
|
||||
"${_JAVA_HOME}/lib/*"
|
||||
"${_JAVA_HOME}/lib"
|
||||
"${_JAVA_HOME}/include/*"
|
||||
"${_JAVA_HOME}/include"
|
||||
"${_JAVA_HOME}"
|
||||
)
|
||||
FIND_PATH(JAVA_INCLUDE_PATH
|
||||
NAMES jni.h
|
||||
PATHS ${_JDK_DIRS}
|
||||
NO_DEFAULT_PATH)
|
||||
#In IBM java, it's jniport.h instead of jni_md.h
|
||||
FIND_PATH(JAVA_INCLUDE_PATH2
|
||||
NAMES jni_md.h jniport.h
|
||||
PATHS ${_JDK_DIRS}
|
||||
NO_DEFAULT_PATH)
|
||||
SET(JNI_INCLUDE_DIRS ${JAVA_INCLUDE_PATH} ${JAVA_INCLUDE_PATH2})
|
||||
FIND_LIBRARY(JAVA_JVM_LIBRARY
|
||||
NAMES rt jvm
|
||||
PATHS ${_JDK_DIRS}
|
||||
NO_DEFAULT_PATH)
|
||||
SET(JNI_LIBRARIES ${JAVA_JVM_LIBRARY})
|
||||
MESSAGE("JAVA_HOME=${JAVA_HOME}, JAVA_JVM_LIBRARY=${JAVA_JVM_LIBRARY}")
|
||||
MESSAGE("JAVA_INCLUDE_PATH=${JAVA_INCLUDE_PATH}, JAVA_INCLUDE_PATH2=${JAVA_INCLUDE_PATH2}")
|
||||
IF(JAVA_JVM_LIBRARY AND JAVA_INCLUDE_PATH AND JAVA_INCLUDE_PATH2)
|
||||
MESSAGE("Located all JNI components successfully.")
|
||||
ELSE()
|
||||
MESSAGE(FATAL_ERROR "Failed to find a viable JVM installation under JAVA_HOME.")
|
||||
ENDIF()
|
||||
ELSE()
|
||||
find_package(JNI REQUIRED)
|
||||
ENDIF()
|
|
@ -0,0 +1,23 @@
|
|||
/**
|
||||
* 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 CONFIG_H
|
||||
#define CONFIG_H
|
||||
|
||||
#cmakedefine HADOOP_SNAPPY_LIBRARY "@HADOOP_SNAPPY_LIBRARY@"
|
||||
|
||||
#endif
|
|
@ -0,0 +1,57 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class Command {
|
||||
|
||||
private int id;
|
||||
private String description;
|
||||
|
||||
public Command(int id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
public Command(int id, String description) {
|
||||
this.id = id;
|
||||
this.description = description;
|
||||
}
|
||||
|
||||
public int id() {
|
||||
return this.id;
|
||||
}
|
||||
|
||||
public String description() {
|
||||
return this.description;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (other instanceof Command) {
|
||||
return this.id == ((Command)other).id;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return id;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,33 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer;
|
||||
|
||||
/**
|
||||
* a CommandDispatcher receives {@link Command} from upstream
|
||||
* and performs corresponding operations
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface CommandDispatcher {
|
||||
public ReadWriteBuffer onCall(Command command, ReadWriteBuffer parameter) throws IOException;
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
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 NATIVE_LOG_DEVICE = "native.log.device";
|
||||
public static final String NATIVE_HADOOP_VERSION = "native.hadoop.version";
|
||||
|
||||
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;
|
||||
|
||||
public static final String NATIVE_STATUS_UPDATE_INTERVAL = "native.update.interval";
|
||||
public static final int NATIVE_STATUS_UPDATE_INTERVAL_DEFVAL = 3000;
|
||||
|
||||
public static final String SERIALIZATION_FRAMEWORK = "SerializationFramework";
|
||||
public static final int SIZEOF_PARTITION_LENGTH = 4;
|
||||
public static final int SIZEOF_KEY_LENGTH = 4;
|
||||
public static final int SIZEOF_VALUE_LENGTH = 4;
|
||||
public static final int SIZEOF_KV_LENGTH = SIZEOF_KEY_LENGTH + SIZEOF_VALUE_LENGTH;
|
||||
|
||||
public static final String NATIVE_CLASS_LIBRARY = "native.class.library";
|
||||
public static final String NATIVE_CLASS_LIBRARY_CUSTOM = "native.class.library.custom";
|
||||
public static final String NATIVE_CLASS_LIBRARY_BUILDIN = "native.class.library.buildin";
|
||||
public static final String NATIVE_MAPOUT_KEY_COMPARATOR = "native.map.output.key.comparator";
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public enum DataChannel {
|
||||
/**
|
||||
* We will only read data from this channel
|
||||
*/
|
||||
IN,
|
||||
/**
|
||||
* We will only write data from this channel
|
||||
*/
|
||||
OUT,
|
||||
/**
|
||||
* We will do both read and write for this channel
|
||||
*/
|
||||
INOUT,
|
||||
/**
|
||||
* There is no data exchange
|
||||
*/
|
||||
NONE
|
||||
}
|
|
@ -0,0 +1,37 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* a DataReceiver pulls in arriving data, an example
|
||||
* is {@link org.apache.hadoop.mapred.nativetask.handlers.BufferPuller}
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface DataReceiver {
|
||||
|
||||
/**
|
||||
* Send a signal to the receiver that the data arrives.
|
||||
* The data is transferred in another band.
|
||||
*/
|
||||
public boolean receiveData() throws IOException;
|
||||
}
|
|
@ -0,0 +1,84 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.BooleanWritable;
|
||||
import org.apache.hadoop.io.ByteWritable;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.DoubleWritable;
|
||||
import org.apache.hadoop.io.FloatWritable;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.VIntWritable;
|
||||
import org.apache.hadoop.io.VLongWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.nativetask.serde.*;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class HadoopPlatform extends Platform {
|
||||
private static final Log LOG = LogFactory.getLog(HadoopPlatform.class);
|
||||
|
||||
public HadoopPlatform() throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init() throws IOException {
|
||||
registerKey(NullWritable.class.getName(), NullWritableSerializer.class);
|
||||
registerKey(Text.class.getName(), TextSerializer.class);
|
||||
registerKey(LongWritable.class.getName(), LongWritableSerializer.class);
|
||||
registerKey(IntWritable.class.getName(), IntWritableSerializer.class);
|
||||
registerKey(Writable.class.getName(), DefaultSerializer.class);
|
||||
registerKey(BytesWritable.class.getName(), BytesWritableSerializer.class);
|
||||
registerKey(BooleanWritable.class.getName(), BoolWritableSerializer.class);
|
||||
registerKey(ByteWritable.class.getName(), ByteWritableSerializer.class);
|
||||
registerKey(FloatWritable.class.getName(), FloatWritableSerializer.class);
|
||||
registerKey(DoubleWritable.class.getName(), DoubleWritableSerializer.class);
|
||||
registerKey(VIntWritable.class.getName(), VIntWritableSerializer.class);
|
||||
registerKey(VLongWritable.class.getName(), VLongWritableSerializer.class);
|
||||
|
||||
LOG.info("Hadoop platform inited");
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean support(String keyClassName, INativeSerializer<?> serializer, JobConf job) {
|
||||
if (keyClassNames.contains(keyClassName)
|
||||
&& serializer instanceof INativeComparable) {
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean define(Class<?> comparatorClass) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return "Hadoop";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* interacts with native side to support Java Combiner
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface ICombineHandler {
|
||||
|
||||
/**
|
||||
* run combiner
|
||||
*/
|
||||
public void combine() throws IOException;
|
||||
|
||||
/**
|
||||
* @return id of this handler
|
||||
*/
|
||||
public long getId();
|
||||
|
||||
/**
|
||||
* close handlers, buffer pullers and pushers
|
||||
*/
|
||||
public void close() throws IOException;
|
||||
}
|
|
@ -0,0 +1,54 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Any key type that is comparable at native side must implement this interface.
|
||||
*
|
||||
* A native comparator function should have the ComparatorPtr type:
|
||||
* <code>
|
||||
* typedef int (*ComparatorPtr)(const char * src, uint32_t srcLength,
|
||||
* const char * dest, uint32_t destLength);
|
||||
* </code>
|
||||
* Keys are in serialized format at native side. The function has passed in
|
||||
* the keys' locations and lengths such that we can compare them in the same
|
||||
* logic as their Java comparator.
|
||||
*
|
||||
* For example, a HiveKey serialized as an int field (containing the length of
|
||||
* raw bytes) + raw bytes.
|
||||
* When comparing two HiveKeys, we first read the length field and then
|
||||
* compare the raw bytes by invoking the BytesComparator provided by our library.
|
||||
* We pass the location and length of raw bytes into BytesComparator.
|
||||
*
|
||||
* <code>
|
||||
* int HivePlatform::HiveKeyComparator(const char * src, uint32_t srcLength,
|
||||
* const char * dest, uint32_t destLength) {
|
||||
* uint32_t sl = bswap(*(uint32_t*)src);
|
||||
* uint32_t dl = bswap(*(uint32_t*)dest);
|
||||
* return NativeObjectFactory::BytesComparator(src + 4, sl, dest + 4, dl);
|
||||
* }
|
||||
* </code>
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public interface INativeComparable {
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer;
|
||||
|
||||
/**
|
||||
* A Handler accept input, and give output can be used to transfer command and data
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface INativeHandler extends NativeDataTarget, NativeDataSource {
|
||||
|
||||
public String name();
|
||||
|
||||
public long getNativeHandler();
|
||||
|
||||
/**
|
||||
* init the native handler
|
||||
*/
|
||||
public void init(Configuration conf) throws IOException;
|
||||
|
||||
/**
|
||||
* close the native handler
|
||||
*/
|
||||
public void close() throws IOException;
|
||||
|
||||
/**
|
||||
* call command to downstream
|
||||
*/
|
||||
public ReadWriteBuffer call(Command command, ReadWriteBuffer parameter) throws IOException;
|
||||
|
||||
void setCommandDispatcher(CommandDispatcher handler);
|
||||
|
||||
}
|
|
@ -0,0 +1,279 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.BufferType;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.InputBuffer;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.OutputBuffer;
|
||||
import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer;
|
||||
import org.apache.hadoop.mapred.nativetask.util.ConfigUtil;
|
||||
|
||||
/**
|
||||
* used to create channel, transfer data and command between Java and native
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class NativeBatchProcessor implements INativeHandler {
|
||||
private static Log LOG = LogFactory.getLog(NativeBatchProcessor.class);
|
||||
|
||||
private final String nativeHandlerName;
|
||||
private long nativeHandlerAddr;
|
||||
|
||||
private boolean isInputFinished = false;
|
||||
|
||||
// << Field used directly in Native, the name must NOT be changed
|
||||
private ByteBuffer rawOutputBuffer;
|
||||
private ByteBuffer rawInputBuffer;
|
||||
// >>
|
||||
|
||||
private InputBuffer in;
|
||||
private OutputBuffer out;
|
||||
|
||||
private CommandDispatcher commandDispatcher;
|
||||
private DataReceiver dataReceiver;
|
||||
|
||||
static {
|
||||
if (NativeRuntime.isNativeLibraryLoaded()) {
|
||||
InitIDs();
|
||||
}
|
||||
}
|
||||
|
||||
public static INativeHandler create(String nativeHandlerName,
|
||||
Configuration conf, DataChannel channel) throws IOException {
|
||||
|
||||
final int bufferSize = conf.getInt(Constants.NATIVE_PROCESSOR_BUFFER_KB,
|
||||
1024) * 1024;
|
||||
|
||||
LOG.info("NativeHandler: direct buffer size: " + bufferSize);
|
||||
|
||||
OutputBuffer out = null;
|
||||
InputBuffer in = null;
|
||||
|
||||
switch (channel) {
|
||||
case IN:
|
||||
in = new InputBuffer(BufferType.DIRECT_BUFFER, bufferSize);
|
||||
break;
|
||||
case OUT:
|
||||
out = new OutputBuffer(BufferType.DIRECT_BUFFER, bufferSize);
|
||||
break;
|
||||
case INOUT:
|
||||
in = new InputBuffer(BufferType.DIRECT_BUFFER, bufferSize);
|
||||
out = new OutputBuffer(BufferType.DIRECT_BUFFER, bufferSize);
|
||||
break;
|
||||
case NONE:
|
||||
}
|
||||
|
||||
final INativeHandler handler = new NativeBatchProcessor(nativeHandlerName,
|
||||
in, out);
|
||||
handler.init(conf);
|
||||
return handler;
|
||||
}
|
||||
|
||||
protected NativeBatchProcessor(String nativeHandlerName, InputBuffer input,
|
||||
OutputBuffer output) throws IOException {
|
||||
this.nativeHandlerName = nativeHandlerName;
|
||||
|
||||
if (null != input) {
|
||||
this.in = input;
|
||||
this.rawInputBuffer = input.getByteBuffer();
|
||||
}
|
||||
if (null != output) {
|
||||
this.out = output;
|
||||
this.rawOutputBuffer = output.getByteBuffer();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setCommandDispatcher(CommandDispatcher handler) {
|
||||
this.commandDispatcher = handler;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(Configuration conf) throws IOException {
|
||||
this.nativeHandlerAddr = NativeRuntime
|
||||
.createNativeObject(nativeHandlerName);
|
||||
if (this.nativeHandlerAddr == 0) {
|
||||
throw new RuntimeException("Native object create failed, class: "
|
||||
+ nativeHandlerName);
|
||||
}
|
||||
setupHandler(nativeHandlerAddr, ConfigUtil.toBytes(conf));
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void close() throws IOException {
|
||||
if (nativeHandlerAddr != 0) {
|
||||
NativeRuntime.releaseNativeObject(nativeHandlerAddr);
|
||||
nativeHandlerAddr = 0;
|
||||
}
|
||||
IOUtils.cleanup(LOG, in);
|
||||
in = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNativeHandler() {
|
||||
return nativeHandlerAddr;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadWriteBuffer call(Command command, ReadWriteBuffer parameter)
|
||||
throws IOException {
|
||||
final byte[] bytes = nativeCommand(nativeHandlerAddr, command.id(),
|
||||
null == parameter ? null : parameter.getBuff());
|
||||
|
||||
final ReadWriteBuffer result = new ReadWriteBuffer(bytes);
|
||||
result.setWritePoint(bytes.length);
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendData() throws IOException {
|
||||
nativeProcessInput(nativeHandlerAddr, rawOutputBuffer.position());
|
||||
rawOutputBuffer.position(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishSendData() throws IOException {
|
||||
if (null == rawOutputBuffer || isInputFinished) {
|
||||
return;
|
||||
}
|
||||
|
||||
sendData();
|
||||
nativeFinish(nativeHandlerAddr);
|
||||
isInputFinished = true;
|
||||
}
|
||||
|
||||
private byte[] sendCommandToJava(int command, byte[] data) throws IOException {
|
||||
try {
|
||||
|
||||
final Command cmd = new Command(command);
|
||||
ReadWriteBuffer param = null;
|
||||
|
||||
if (null != data) {
|
||||
param = new ReadWriteBuffer();
|
||||
param.reset(data);
|
||||
param.setWritePoint(data.length);
|
||||
}
|
||||
|
||||
if (null != commandDispatcher) {
|
||||
ReadWriteBuffer result = null;
|
||||
|
||||
result = commandDispatcher.onCall(cmd, param);
|
||||
if (null != result) {
|
||||
return result.getBuff();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by native side, clean output buffer so native side can continue
|
||||
* processing
|
||||
*/
|
||||
private void flushOutput(int length) throws IOException {
|
||||
|
||||
if (null != rawInputBuffer) {
|
||||
rawInputBuffer.position(0);
|
||||
rawInputBuffer.limit(length);
|
||||
|
||||
if (null != dataReceiver) {
|
||||
try {
|
||||
dataReceiver.receiveData();
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Cache JNI field & method ids
|
||||
*/
|
||||
private static native void InitIDs();
|
||||
|
||||
/**
|
||||
* Setup native side BatchHandler
|
||||
*/
|
||||
private native void setupHandler(long nativeHandlerAddr, byte[][] configs);
|
||||
|
||||
/**
|
||||
* Let native side to process data in inputBuffer
|
||||
*/
|
||||
private native void nativeProcessInput(long handler, int length);
|
||||
|
||||
/**
|
||||
* Notice native side input is finished
|
||||
*/
|
||||
private native void nativeFinish(long handler);
|
||||
|
||||
/**
|
||||
* Send control message to native side
|
||||
*/
|
||||
private native byte[] nativeCommand(long handler, int cmd, byte[] parameter);
|
||||
|
||||
/**
|
||||
* Load data from native
|
||||
*/
|
||||
private native void nativeLoadData(long handler);
|
||||
|
||||
protected void finishOutput() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputBuffer getInputBuffer() {
|
||||
return this.in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OutputBuffer getOutputBuffer() {
|
||||
return this.out;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void loadData() throws IOException {
|
||||
nativeLoadData(nativeHandlerAddr);
|
||||
//
|
||||
// return call(Command.CMD_LOAD, param);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setDataReceiver(DataReceiver handler) {
|
||||
this.dataReceiver = handler;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return nativeHandlerName;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.InputBuffer;
|
||||
|
||||
/**
|
||||
* NativeDataSource loads data from upstream
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface NativeDataSource {
|
||||
|
||||
/**
|
||||
* get input buffer
|
||||
*/
|
||||
public InputBuffer getInputBuffer();
|
||||
|
||||
/**
|
||||
* set listener. When data from upstream arrives, the listener will be activated.
|
||||
*/
|
||||
void setDataReceiver(DataReceiver handler);
|
||||
|
||||
/**
|
||||
* load data from upstream
|
||||
*/
|
||||
public void loadData() throws IOException;
|
||||
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.OutputBuffer;
|
||||
|
||||
/**
|
||||
* NativeDataTarge sends data to downstream
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface NativeDataTarget {
|
||||
|
||||
/**
|
||||
* Sends a signal to indicate that the data has been stored in output buffer
|
||||
*/
|
||||
public void sendData() throws IOException;
|
||||
|
||||
/**
|
||||
* Sends a signal that there is no more data
|
||||
*/
|
||||
public void finishSendData() throws IOException;
|
||||
|
||||
/**
|
||||
* Gets the output buffer.
|
||||
*/
|
||||
public OutputBuffer getOutputBuffer();
|
||||
|
||||
}
|
|
@ -0,0 +1,171 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.RawComparator;
|
||||
import org.apache.hadoop.mapred.InvalidJobConfException;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.MapOutputCollector;
|
||||
import org.apache.hadoop.mapred.TaskAttemptID;
|
||||
import org.apache.hadoop.mapred.nativetask.handlers.NativeCollectorOnlyHandler;
|
||||
import org.apache.hadoop.mapred.nativetask.serde.INativeSerializer;
|
||||
import org.apache.hadoop.mapred.nativetask.serde.NativeSerialization;
|
||||
import org.apache.hadoop.mapreduce.MRConfig;
|
||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.TaskCounter;
|
||||
import org.apache.hadoop.util.QuickSort;
|
||||
|
||||
/**
|
||||
* native map output collector wrapped in Java interface
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class NativeMapOutputCollectorDelegator<K, V> implements MapOutputCollector<K, V> {
|
||||
|
||||
private static Log LOG = LogFactory.getLog(NativeMapOutputCollectorDelegator.class);
|
||||
private JobConf job;
|
||||
private NativeCollectorOnlyHandler<K, V> handler;
|
||||
|
||||
private Context context;
|
||||
private StatusReportChecker updater;
|
||||
|
||||
@Override
|
||||
public void collect(K key, V value, int partition) throws IOException, InterruptedException {
|
||||
handler.collect(key, value, partition);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException, InterruptedException {
|
||||
handler.close();
|
||||
if (null != updater) {
|
||||
updater.stop();
|
||||
NativeRuntime.reportStatus(context.getReporter());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush() throws IOException, InterruptedException, ClassNotFoundException {
|
||||
handler.flush();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void init(Context context) throws IOException, ClassNotFoundException {
|
||||
this.context = context;
|
||||
this.job = context.getJobConf();
|
||||
|
||||
Platforms.init(job);
|
||||
|
||||
if (job.getNumReduceTasks() == 0) {
|
||||
String message = "There is no reducer, no need to use native output collector";
|
||||
LOG.error(message);
|
||||
throw new InvalidJobConfException(message);
|
||||
}
|
||||
|
||||
Class<?> comparatorClass = job.getClass(MRJobConfig.KEY_COMPARATOR, null,
|
||||
RawComparator.class);
|
||||
if (comparatorClass != null && !Platforms.define(comparatorClass)) {
|
||||
String message = "Native output collector doesn't support customized java comparator "
|
||||
+ job.get(MRJobConfig.KEY_COMPARATOR);
|
||||
LOG.error(message);
|
||||
throw new InvalidJobConfException(message);
|
||||
}
|
||||
|
||||
|
||||
|
||||
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);
|
||||
LOG.error(message);
|
||||
throw new InvalidJobConfException(message);
|
||||
}
|
||||
|
||||
if (job.getBoolean(MRConfig.SHUFFLE_SSL_ENABLED_KEY, false) == true) {
|
||||
String message = "Native-Task doesn't support secure shuffle";
|
||||
LOG.error(message);
|
||||
throw new InvalidJobConfException(message);
|
||||
}
|
||||
|
||||
final Class<?> keyCls = job.getMapOutputKeyClass();
|
||||
try {
|
||||
@SuppressWarnings("rawtypes")
|
||||
final INativeSerializer serializer = NativeSerialization.getInstance().getSerializer(keyCls);
|
||||
if (null == serializer) {
|
||||
String message = "Key type not supported. Cannot find serializer for " + keyCls.getName();
|
||||
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();
|
||||
LOG.error(message);
|
||||
throw new InvalidJobConfException(message);
|
||||
}
|
||||
} catch (final IOException e) {
|
||||
String message = "Cannot find serializer for " + keyCls.getName();
|
||||
LOG.error(message);
|
||||
throw new IOException(message);
|
||||
}
|
||||
|
||||
final boolean ret = NativeRuntime.isNativeLibraryLoaded();
|
||||
if (ret) {
|
||||
if (job.getBoolean(MRJobConfig.MAP_OUTPUT_COMPRESS, false)) {
|
||||
String codec = job.get(MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC);
|
||||
if (!NativeRuntime.supportsCompressionCodec(codec.getBytes(Charsets.UTF_8))) {
|
||||
String message = "Native output collector doesn't support compression codec " + codec;
|
||||
LOG.error(message);
|
||||
throw new InvalidJobConfException(message);
|
||||
}
|
||||
}
|
||||
NativeRuntime.configure(job);
|
||||
|
||||
final long updateInterval = job.getLong(Constants.NATIVE_STATUS_UPDATE_INTERVAL,
|
||||
Constants.NATIVE_STATUS_UPDATE_INTERVAL_DEFVAL);
|
||||
updater = new StatusReportChecker(context.getReporter(), updateInterval);
|
||||
updater.start();
|
||||
|
||||
} else {
|
||||
String message = "NativeRuntime cannot be loaded, please check that " +
|
||||
"libnativetask.so is in hadoop library dir";
|
||||
LOG.error(message);
|
||||
throw new InvalidJobConfException(message);
|
||||
}
|
||||
|
||||
this.handler = null;
|
||||
try {
|
||||
final Class<K> oKClass = (Class<K>) job.getMapOutputKeyClass();
|
||||
final Class<K> oVClass = (Class<K>) job.getMapOutputValueClass();
|
||||
final TaskAttemptID id = context.getMapTask().getTaskID();
|
||||
final TaskContext taskContext = new TaskContext(job, null, null, oKClass, oVClass,
|
||||
context.getReporter(), id);
|
||||
handler = NativeCollectorOnlyHandler.create(taskContext);
|
||||
} catch (final IOException e) {
|
||||
String message = "Native output collector cannot be loaded;";
|
||||
LOG.error(message);
|
||||
throw new IOException(message, e);
|
||||
}
|
||||
|
||||
LOG.info("Native output collector can be successfully enabled!");
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,197 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.apache.hadoop.io.FloatWritable;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapred.Task.TaskReporter;
|
||||
import org.apache.hadoop.mapred.nativetask.util.ConfigUtil;
|
||||
import org.apache.hadoop.util.VersionInfo;
|
||||
|
||||
/**
|
||||
* This class stands for the native runtime It has three functions:
|
||||
* 1. Create native handlers for map, reduce, outputcollector, etc
|
||||
* 2. Configure native task with provided MR configs
|
||||
* 3. Provide file system api to native space, so that it can use File system like HDFS.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class NativeRuntime {
|
||||
private static Log LOG = LogFactory.getLog(NativeRuntime.class);
|
||||
private static boolean nativeLibraryLoaded = false;
|
||||
|
||||
private static Configuration conf = new Configuration();
|
||||
|
||||
static {
|
||||
try {
|
||||
System.loadLibrary("nativetask");
|
||||
LOG.info("Nativetask JNI library loaded.");
|
||||
nativeLibraryLoaded = true;
|
||||
} catch (final Throwable t) {
|
||||
// Ignore failures
|
||||
LOG.error("Failed to load nativetask JNI library with error: " + t);
|
||||
LOG.info("java.library.path=" + System.getProperty("java.library.path"));
|
||||
LOG.info("LD_LIBRARY_PATH=" + System.getenv("LD_LIBRARY_PATH"));
|
||||
}
|
||||
}
|
||||
|
||||
private static void assertNativeLibraryLoaded() {
|
||||
if (!nativeLibraryLoaded) {
|
||||
throw new RuntimeException("Native runtime library not loaded");
|
||||
}
|
||||
}
|
||||
|
||||
public static boolean isNativeLibraryLoaded() {
|
||||
return nativeLibraryLoaded;
|
||||
}
|
||||
|
||||
public static void configure(Configuration jobConf) {
|
||||
assertNativeLibraryLoaded();
|
||||
conf = new Configuration(jobConf);
|
||||
conf.set(Constants.NATIVE_HADOOP_VERSION, VersionInfo.getVersion());
|
||||
JNIConfigure(ConfigUtil.toBytes(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
* create native object We use it to create native handlers
|
||||
*/
|
||||
public synchronized static long createNativeObject(String clazz) {
|
||||
assertNativeLibraryLoaded();
|
||||
final long ret = JNICreateNativeObject(clazz.getBytes(Charsets.UTF_8));
|
||||
if (ret == 0) {
|
||||
LOG.warn("Can't create NativeObject for class " + clazz + ", probably not exist.");
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
/**
|
||||
* Register a customized library
|
||||
*/
|
||||
public synchronized static long registerLibrary(String libraryName, String clazz) {
|
||||
assertNativeLibraryLoaded();
|
||||
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.");
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
/**
|
||||
* destroy native object We use to destory native handlers
|
||||
*/
|
||||
public synchronized static void releaseNativeObject(long addr) {
|
||||
assertNativeLibraryLoaded();
|
||||
JNIReleaseNativeObject(addr);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the status report from native space
|
||||
*/
|
||||
public static void reportStatus(TaskReporter reporter) throws IOException {
|
||||
assertNativeLibraryLoaded();
|
||||
synchronized (reporter) {
|
||||
final byte[] statusBytes = JNIUpdateStatus();
|
||||
final DataInputBuffer ib = new DataInputBuffer();
|
||||
ib.reset(statusBytes, statusBytes.length);
|
||||
final FloatWritable progress = new FloatWritable();
|
||||
progress.readFields(ib);
|
||||
reporter.setProgress(progress.get());
|
||||
final Text status = new Text();
|
||||
status.readFields(ib);
|
||||
if (status.getLength() > 0) {
|
||||
reporter.setStatus(status.toString());
|
||||
}
|
||||
final IntWritable numCounters = new IntWritable();
|
||||
numCounters.readFields(ib);
|
||||
if (numCounters.get() == 0) {
|
||||
return;
|
||||
}
|
||||
final Text group = new Text();
|
||||
final Text name = new Text();
|
||||
final LongWritable amount = new LongWritable();
|
||||
for (int i = 0; i < numCounters.get(); i++) {
|
||||
group.readFields(ib);
|
||||
name.readFields(ib);
|
||||
amount.readFields(ib);
|
||||
reporter.incrCounter(group.toString(), name.toString(), amount.get());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*******************************************************
|
||||
*** The following are JNI Apis
|
||||
********************************************************/
|
||||
|
||||
/**
|
||||
* Check whether the native side has compression codec support built in
|
||||
*/
|
||||
public native static boolean supportsCompressionCodec(byte[] codec);
|
||||
|
||||
/**
|
||||
* Config the native runtime with mapreduce job configurations.
|
||||
*/
|
||||
private native static void JNIConfigure(byte[][] configs);
|
||||
|
||||
/**
|
||||
* create a native object in native space
|
||||
*/
|
||||
private native static long JNICreateNativeObject(byte[] clazz);
|
||||
|
||||
/**
|
||||
* create the default native object for certain type
|
||||
*/
|
||||
@Deprecated
|
||||
private native static long JNICreateDefaultNativeObject(byte[] type);
|
||||
|
||||
/**
|
||||
* destroy native object in native space
|
||||
*/
|
||||
private native static void JNIReleaseNativeObject(long addr);
|
||||
|
||||
/**
|
||||
* 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();
|
||||
|
||||
/**
|
||||
* Not used.
|
||||
*/
|
||||
private native static void JNIRelease();
|
||||
|
||||
/**
|
||||
* Not used.
|
||||
*/
|
||||
private native static int JNIRegisterModule(byte[] path, byte[] name);
|
||||
}
|
|
@ -0,0 +1,100 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.nativetask.serde.INativeSerializer;
|
||||
import org.apache.hadoop.mapred.nativetask.serde.NativeSerialization;
|
||||
|
||||
/**
|
||||
* Base class for platforms. A platform is a framework running on top of
|
||||
* MapReduce, like Hadoop, Hive, Pig, Mahout. Each framework defines its
|
||||
* own key type and value type across a MapReduce job. For each platform,
|
||||
* we should implement serializers such that we could communicate data with
|
||||
* native side and native comparators so our native output collectors could
|
||||
* sort them and write out. We've already provided the {@link HadoopPlatform}
|
||||
* that supports all key types of Hadoop and users could implement their custom
|
||||
* platform.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public abstract class Platform {
|
||||
private final NativeSerialization serialization;
|
||||
protected Set<String> keyClassNames = new HashSet<String>();
|
||||
|
||||
public Platform() {
|
||||
this.serialization = NativeSerialization.getInstance();
|
||||
}
|
||||
|
||||
/**
|
||||
* initialize a platform, where we should call registerKey
|
||||
*/
|
||||
public abstract void init() throws IOException;
|
||||
|
||||
/**
|
||||
* @return name of a Platform, useful for logs and debug
|
||||
*/
|
||||
public abstract String name();
|
||||
|
||||
|
||||
/**
|
||||
* associate a key class with its serializer and platform
|
||||
*
|
||||
* @param keyClassName map out key class name
|
||||
* @param key key serializer class
|
||||
*/
|
||||
protected void registerKey(String keyClassName, Class<?> key) throws IOException {
|
||||
serialization.register(keyClassName, key);
|
||||
keyClassNames.add(keyClassName);
|
||||
}
|
||||
|
||||
/**
|
||||
* whether a platform supports a specific key should at least satisfy two conditions
|
||||
*
|
||||
* 1. the key belongs to the platform
|
||||
* 2. the associated serializer must implement {@link INativeComparable} interface
|
||||
*
|
||||
*
|
||||
* @param keyClassName map out put key class name
|
||||
* @param serializer serializer associated with key via registerKey
|
||||
* @param job job configuration
|
||||
* @return true if the platform has implemented native comparators of the key and
|
||||
* false otherwise
|
||||
*/
|
||||
protected abstract boolean support(String keyClassName,
|
||||
INativeSerializer<?> serializer, JobConf job);
|
||||
|
||||
|
||||
/**
|
||||
* whether it's the platform that has defined a custom Java comparator
|
||||
*
|
||||
* 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
|
||||
*/
|
||||
protected abstract boolean define(Class<?> keyComparator);
|
||||
}
|
|
@ -0,0 +1,79 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ServiceLoader;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.nativetask.serde.INativeSerializer;
|
||||
import org.apache.hadoop.mapred.nativetask.serde.NativeSerialization;
|
||||
|
||||
|
||||
/**
|
||||
* this class will load in and init all platforms on classpath
|
||||
* it is also the facade to check for key type support and other
|
||||
* platform methods
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class Platforms {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(Platforms.class);
|
||||
private static final ServiceLoader<Platform> platforms = ServiceLoader.load(Platform.class);
|
||||
|
||||
public static void init(Configuration conf) throws IOException {
|
||||
|
||||
NativeSerialization.getInstance().reset();
|
||||
synchronized (platforms) {
|
||||
for (Platform platform : platforms) {
|
||||
platform.init();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static boolean support(String keyClassName,
|
||||
INativeSerializer<?> serializer, JobConf job) {
|
||||
synchronized (platforms) {
|
||||
for (Platform platform : platforms) {
|
||||
if (platform.support(keyClassName, serializer, job)) {
|
||||
LOG.debug("platform " + platform.name() + " support key class"
|
||||
+ keyClassName);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public static boolean define(Class<?> keyComparator) {
|
||||
synchronized (platforms) {
|
||||
for (Platform platform : platforms) {
|
||||
if (platform.define(keyComparator)) {
|
||||
LOG.debug("platform " + platform.name() + " define comparator "
|
||||
+ keyComparator.getName());
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,99 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.mapred.Task.TaskReporter;
|
||||
import org.apache.hadoop.mapreduce.TaskCounter;
|
||||
import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
|
||||
|
||||
/**
|
||||
* Will periodically check status from native and report to MR framework.
|
||||
*
|
||||
*/
|
||||
class StatusReportChecker implements Runnable {
|
||||
|
||||
private static Log LOG = LogFactory.getLog(StatusReportChecker.class);
|
||||
public static final int INTERVAL = 1000; // milliseconds
|
||||
|
||||
private Thread checker;
|
||||
private final TaskReporter reporter;
|
||||
private final long interval;
|
||||
|
||||
public StatusReportChecker(TaskReporter reporter) {
|
||||
this(reporter, INTERVAL);
|
||||
}
|
||||
|
||||
public StatusReportChecker(TaskReporter reporter, long interval) {
|
||||
this.reporter = reporter;
|
||||
this.interval = interval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
while (true) {
|
||||
try {
|
||||
Thread.sleep(interval);
|
||||
} catch (final InterruptedException e) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("StatusUpdater thread exiting " + "since it got interrupted");
|
||||
}
|
||||
break;
|
||||
}
|
||||
try {
|
||||
NativeRuntime.reportStatus(reporter);
|
||||
} catch (final IOException e) {
|
||||
LOG.warn("Update native status got exception", e);
|
||||
reporter.setStatus(e.toString());
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected void initUsedCounters() {
|
||||
reporter.getCounter(TaskCounter.MAP_INPUT_RECORDS);
|
||||
reporter.getCounter(TaskCounter.MAP_OUTPUT_RECORDS);
|
||||
reporter.getCounter(FileInputFormatCounter.BYTES_READ);
|
||||
reporter.getCounter(TaskCounter.MAP_OUTPUT_BYTES);
|
||||
reporter.getCounter(TaskCounter.MAP_OUTPUT_MATERIALIZED_BYTES);
|
||||
reporter.getCounter(TaskCounter.COMBINE_INPUT_RECORDS);
|
||||
reporter.getCounter(TaskCounter.COMBINE_OUTPUT_RECORDS);
|
||||
reporter.getCounter(TaskCounter.SPILLED_RECORDS);
|
||||
}
|
||||
|
||||
public synchronized void start() {
|
||||
if (checker == null) {
|
||||
// init counters used by native side,
|
||||
// so they will have correct display name
|
||||
initUsedCounters();
|
||||
checker = new Thread(this);
|
||||
checker.setDaemon(true);
|
||||
checker.start();
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized void stop() throws InterruptedException {
|
||||
if (checker != null) {
|
||||
checker.interrupt();
|
||||
checker.join();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,94 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.Task.TaskReporter;
|
||||
import org.apache.hadoop.mapred.TaskAttemptID;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class TaskContext {
|
||||
private final JobConf conf;
|
||||
private Class<?> iKClass;
|
||||
private Class<?> iVClass;
|
||||
private Class<?> oKClass;
|
||||
private Class<?> oVClass;
|
||||
private final TaskReporter reporter;
|
||||
private final TaskAttemptID taskAttemptID;
|
||||
|
||||
public TaskContext(JobConf conf, Class<?> iKClass, Class<?> iVClass,
|
||||
Class<?> oKClass, Class<?> oVClass, TaskReporter reporter,
|
||||
TaskAttemptID id) {
|
||||
this.conf = conf;
|
||||
this.iKClass = iKClass;
|
||||
this.iVClass = iVClass;
|
||||
this.oKClass = oKClass;
|
||||
this.oVClass = oVClass;
|
||||
this.reporter = reporter;
|
||||
this.taskAttemptID = id;
|
||||
}
|
||||
|
||||
public Class<?> getInputKeyClass() {
|
||||
return iKClass;
|
||||
}
|
||||
|
||||
public void setInputKeyClass(Class<?> klass) {
|
||||
this.iKClass = klass;
|
||||
}
|
||||
|
||||
public Class<?> getInputValueClass() {
|
||||
return iVClass;
|
||||
}
|
||||
|
||||
public void setInputValueClass(Class<?> klass) {
|
||||
this.iVClass = klass;
|
||||
}
|
||||
|
||||
public Class<?> getOutputKeyClass() {
|
||||
return this.oKClass;
|
||||
}
|
||||
|
||||
public void setOutputKeyClass(Class<?> klass) {
|
||||
this.oKClass = klass;
|
||||
}
|
||||
|
||||
public Class<?> getOutputValueClass() {
|
||||
return this.oVClass;
|
||||
}
|
||||
|
||||
public void setOutputValueClass(Class<?> klass) {
|
||||
this.oVClass = klass;
|
||||
}
|
||||
|
||||
public TaskReporter getTaskReporter() {
|
||||
return this.reporter;
|
||||
}
|
||||
|
||||
public TaskAttemptID getTaskAttemptId() {
|
||||
return this.taskAttemptID;
|
||||
}
|
||||
|
||||
public JobConf getConf() {
|
||||
return this.conf;
|
||||
}
|
||||
|
||||
public TaskContext copyOf() {
|
||||
return new TaskContext(conf, iKClass, iVClass, oKClass, oVClass, reporter, taskAttemptID);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,27 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.buffer;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public enum BufferType {
|
||||
DIRECT_BUFFER,
|
||||
HEAP_BUFFER
|
||||
};
|
|
@ -0,0 +1,148 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.buffer;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* read data from a input buffer
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ByteBufferDataReader extends DataInputStream {
|
||||
private ByteBuffer byteBuffer;
|
||||
private java.io.DataInputStream javaReader;
|
||||
|
||||
public ByteBufferDataReader(InputBuffer buffer) {
|
||||
if (buffer != null) {
|
||||
reset(buffer);
|
||||
}
|
||||
javaReader = new java.io.DataInputStream(this);
|
||||
}
|
||||
|
||||
public void reset(InputBuffer buffer) {
|
||||
this.byteBuffer = buffer.getByteBuffer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
return byteBuffer.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(byte b[], int off, int len) throws IOException {
|
||||
byteBuffer.get(b, off, len);
|
||||
return len;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFully(byte[] b) throws IOException {
|
||||
byteBuffer.get(b, 0, b.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFully(byte[] b, int off, int len) throws IOException {
|
||||
byteBuffer.get(b, off, len);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int skipBytes(int n) throws IOException {
|
||||
final int remains = byteBuffer.remaining();
|
||||
final int skip = (remains < n) ? remains : n;
|
||||
final int current = byteBuffer.position();
|
||||
byteBuffer.position(current + skip);
|
||||
return skip;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean readBoolean() throws IOException {
|
||||
return (byteBuffer.get() == 1) ? true : false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte readByte() throws IOException {
|
||||
return byteBuffer.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readUnsignedByte() throws IOException {
|
||||
final int ch = byteBuffer.get();
|
||||
if (ch < 0) {
|
||||
throw new EOFException();
|
||||
}
|
||||
return ch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public short readShort() throws IOException {
|
||||
return byteBuffer.getShort();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readUnsignedShort() throws IOException {
|
||||
return byteBuffer.getShort();
|
||||
}
|
||||
|
||||
@Override
|
||||
public char readChar() throws IOException {
|
||||
return byteBuffer.getChar();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readInt() throws IOException {
|
||||
return byteBuffer.getInt();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long readLong() throws IOException {
|
||||
return byteBuffer.getLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float readFloat() throws IOException {
|
||||
return byteBuffer.getFloat();
|
||||
}
|
||||
|
||||
@Override
|
||||
public double readDouble() throws IOException {
|
||||
return byteBuffer.getDouble();
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
@Override
|
||||
public String readLine() throws IOException {
|
||||
return javaReader.readLine();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final String readUTF() throws IOException {
|
||||
return javaReader.readUTF();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasUnReadData() {
|
||||
return null != byteBuffer && byteBuffer.hasRemaining();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,169 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.buffer;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.mapred.nativetask.NativeDataTarget;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* DataOutputStream implementation which buffers data in a fixed-size
|
||||
* ByteBuffer.
|
||||
* When the byte buffer has filled up, synchronously passes the buffer
|
||||
* to a downstream NativeDataTarget.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ByteBufferDataWriter extends DataOutputStream {
|
||||
private final ByteBuffer buffer;
|
||||
private final NativeDataTarget target;
|
||||
|
||||
private final static byte TRUE = (byte) 1;
|
||||
private final static byte FALSE = (byte) 0;
|
||||
private final java.io.DataOutputStream javaWriter;
|
||||
|
||||
private void checkSizeAndFlushIfNecessary(int length) throws IOException {
|
||||
if (buffer.position() > 0 && buffer.remaining() < length) {
|
||||
flush();
|
||||
}
|
||||
}
|
||||
|
||||
public ByteBufferDataWriter(NativeDataTarget handler) {
|
||||
Preconditions.checkNotNull(handler);
|
||||
this.buffer = handler.getOutputBuffer().getByteBuffer();
|
||||
this.target = handler;
|
||||
this.javaWriter = new java.io.DataOutputStream(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void write(int v) throws IOException {
|
||||
checkSizeAndFlushIfNecessary(1);
|
||||
buffer.put((byte) v);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean shortOfSpace(int dataLength) throws IOException {
|
||||
if (buffer.remaining() < dataLength) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void write(byte b[], int off, int len) throws IOException {
|
||||
int remain = len;
|
||||
int offset = off;
|
||||
while (remain > 0) {
|
||||
int currentFlush = 0;
|
||||
if (buffer.remaining() > 0) {
|
||||
currentFlush = Math.min(buffer.remaining(), remain);
|
||||
buffer.put(b, offset, currentFlush);
|
||||
remain -= currentFlush;
|
||||
offset += currentFlush;
|
||||
} else {
|
||||
flush();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush() throws IOException {
|
||||
target.sendData();
|
||||
buffer.position(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (hasUnFlushedData()) {
|
||||
flush();
|
||||
}
|
||||
target.finishSendData();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void writeBoolean(boolean v) throws IOException {
|
||||
checkSizeAndFlushIfNecessary(1);
|
||||
buffer.put(v ? TRUE : FALSE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void writeByte(int v) throws IOException {
|
||||
checkSizeAndFlushIfNecessary(1);
|
||||
buffer.put((byte) v);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void writeShort(int v) throws IOException {
|
||||
checkSizeAndFlushIfNecessary(2);
|
||||
buffer.putShort((short) v);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void writeChar(int v) throws IOException {
|
||||
checkSizeAndFlushIfNecessary(2);
|
||||
buffer.put((byte) ((v >>> 8) & 0xFF));
|
||||
buffer.put((byte) ((v >>> 0) & 0xFF));
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void writeInt(int v) throws IOException {
|
||||
checkSizeAndFlushIfNecessary(4);
|
||||
buffer.putInt(v);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void writeLong(long v) throws IOException {
|
||||
checkSizeAndFlushIfNecessary(8);
|
||||
buffer.putLong(v);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void writeFloat(float v) throws IOException {
|
||||
checkSizeAndFlushIfNecessary(4);
|
||||
writeInt(Float.floatToIntBits(v));
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void writeDouble(double v) throws IOException {
|
||||
checkSizeAndFlushIfNecessary(8);
|
||||
writeLong(Double.doubleToLongBits(v));
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void writeBytes(String s) throws IOException {
|
||||
javaWriter.writeBytes(s);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void writeChars(String s) throws IOException {
|
||||
javaWriter.writeChars(s);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void writeUTF(String str) throws IOException {
|
||||
javaWriter.writeUTF(str);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasUnFlushedData() {
|
||||
return buffer.position() > 0;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,27 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.buffer;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.InputStream;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public abstract class DataInputStream extends InputStream implements DataInput {
|
||||
public abstract boolean hasUnReadData();
|
||||
}
|
|
@ -0,0 +1,39 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.buffer;
|
||||
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public abstract class DataOutputStream extends OutputStream implements DataOutput {
|
||||
/**
|
||||
* Check whether this buffer has enough space to store length of bytes
|
||||
*
|
||||
* @param length length of bytes
|
||||
*/
|
||||
public abstract boolean shortOfSpace(int length) throws IOException;
|
||||
|
||||
/**
|
||||
* Check whether there is unflushed data stored in the stream
|
||||
*/
|
||||
public abstract boolean hasUnFlushedData();
|
||||
}
|
|
@ -0,0 +1,136 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.buffer;
|
||||
|
||||
import org.apache.hadoop.util.DirectBufferPool;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class InputBuffer implements Closeable {
|
||||
|
||||
static DirectBufferPool bufferPool = new DirectBufferPool();
|
||||
|
||||
private ByteBuffer byteBuffer;
|
||||
private final BufferType type;
|
||||
|
||||
public InputBuffer(BufferType type, int inputSize) throws IOException {
|
||||
|
||||
final int capacity = inputSize;
|
||||
this.type = type;
|
||||
|
||||
if (capacity > 0) {
|
||||
|
||||
switch (type) {
|
||||
case DIRECT_BUFFER:
|
||||
this.byteBuffer = bufferPool.getBuffer(capacity);
|
||||
this.byteBuffer.order(ByteOrder.BIG_ENDIAN);
|
||||
break;
|
||||
case HEAP_BUFFER:
|
||||
this.byteBuffer = ByteBuffer.allocate(capacity);
|
||||
this.byteBuffer.order(ByteOrder.BIG_ENDIAN);
|
||||
break;
|
||||
}
|
||||
byteBuffer.position(0);
|
||||
byteBuffer.limit(0);
|
||||
}
|
||||
}
|
||||
|
||||
public BufferType getType() {
|
||||
return this.type;
|
||||
}
|
||||
|
||||
public InputBuffer(byte[] bytes) {
|
||||
this.type = BufferType.HEAP_BUFFER;
|
||||
if (bytes.length > 0) {
|
||||
this.byteBuffer = ByteBuffer.wrap(bytes);
|
||||
this.byteBuffer.order(ByteOrder.BIG_ENDIAN);
|
||||
byteBuffer.position(0);
|
||||
byteBuffer.limit(0);
|
||||
}
|
||||
}
|
||||
|
||||
public ByteBuffer getByteBuffer() {
|
||||
return this.byteBuffer;
|
||||
}
|
||||
|
||||
public int length() {
|
||||
if (null == byteBuffer) {
|
||||
return 0;
|
||||
}
|
||||
return byteBuffer.limit();
|
||||
}
|
||||
|
||||
public void rewind(int startOffset, int length) {
|
||||
if (null == byteBuffer) {
|
||||
return;
|
||||
}
|
||||
byteBuffer.position(startOffset);
|
||||
byteBuffer.limit(length);
|
||||
}
|
||||
|
||||
public int remaining() {
|
||||
if (null == byteBuffer) {
|
||||
return 0;
|
||||
}
|
||||
return byteBuffer.remaining();
|
||||
}
|
||||
|
||||
public int position() {
|
||||
if (null == byteBuffer) {
|
||||
return 0;
|
||||
}
|
||||
return byteBuffer.position();
|
||||
}
|
||||
|
||||
public int position(int pos) {
|
||||
if (null == byteBuffer) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
byteBuffer.position(pos);
|
||||
return pos;
|
||||
}
|
||||
|
||||
public int capacity() {
|
||||
if (null == byteBuffer) {
|
||||
return 0;
|
||||
}
|
||||
return byteBuffer.capacity();
|
||||
}
|
||||
|
||||
public byte[] array() {
|
||||
if (null == byteBuffer) {
|
||||
return null;
|
||||
}
|
||||
return byteBuffer.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (byteBuffer != null && byteBuffer.isDirect()) {
|
||||
bufferPool.returnBuffer(byteBuffer);
|
||||
byteBuffer = null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,76 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.buffer;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class OutputBuffer {
|
||||
protected ByteBuffer byteBuffer;
|
||||
private final BufferType type;
|
||||
|
||||
public OutputBuffer(BufferType type, int outputBufferCapacity) {
|
||||
|
||||
this.type = type;
|
||||
if (outputBufferCapacity > 0) {
|
||||
switch (type) {
|
||||
case DIRECT_BUFFER:
|
||||
this.byteBuffer = ByteBuffer.allocateDirect(outputBufferCapacity);
|
||||
this.byteBuffer.order(ByteOrder.BIG_ENDIAN);
|
||||
break;
|
||||
case HEAP_BUFFER:
|
||||
this.byteBuffer = ByteBuffer.allocate(outputBufferCapacity);
|
||||
this.byteBuffer.order(ByteOrder.BIG_ENDIAN);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public OutputBuffer(byte[] bytes) {
|
||||
this.type = BufferType.HEAP_BUFFER;
|
||||
final int outputBufferCapacity = bytes.length;
|
||||
if (outputBufferCapacity > 0) {
|
||||
this.byteBuffer = ByteBuffer.wrap(bytes);
|
||||
this.byteBuffer.order(ByteOrder.BIG_ENDIAN);
|
||||
this.byteBuffer.position(0);
|
||||
}
|
||||
}
|
||||
|
||||
public BufferType getType() {
|
||||
return this.type;
|
||||
}
|
||||
|
||||
public ByteBuffer getByteBuffer() {
|
||||
return this.byteBuffer;
|
||||
}
|
||||
|
||||
public int length() {
|
||||
return byteBuffer.position();
|
||||
}
|
||||
|
||||
public void rewind() {
|
||||
byteBuffer.position(0);
|
||||
}
|
||||
|
||||
public int limit() {
|
||||
return byteBuffer.limit();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,121 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.handlers;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.mapred.RawKeyValueIterator;
|
||||
import org.apache.hadoop.mapred.nativetask.Constants;
|
||||
import org.apache.hadoop.mapred.nativetask.NativeDataTarget;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.ByteBufferDataWriter;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.OutputBuffer;
|
||||
import org.apache.hadoop.mapred.nativetask.serde.KVSerializer;
|
||||
import org.apache.hadoop.mapred.nativetask.util.SizedWritable;
|
||||
|
||||
/**
|
||||
* load data into a buffer signaled by a {@link BufferPuller}
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class BufferPullee<IK, IV> implements IDataLoader {
|
||||
|
||||
public static final int KV_HEADER_LENGTH = Constants.SIZEOF_KV_LENGTH;
|
||||
|
||||
private final SizedWritable<IK> tmpInputKey;
|
||||
private final SizedWritable<IV> tmpInputValue;
|
||||
private boolean inputKVBufferd = false;
|
||||
private RawKeyValueIterator rIter;
|
||||
private ByteBufferDataWriter nativeWriter;
|
||||
protected KVSerializer<IK, IV> serializer;
|
||||
private final OutputBuffer outputBuffer;
|
||||
private final NativeDataTarget target;
|
||||
private boolean closed = false;
|
||||
|
||||
public BufferPullee(Class<IK> iKClass, Class<IV> iVClass,
|
||||
RawKeyValueIterator rIter, NativeDataTarget target)
|
||||
throws IOException {
|
||||
this.rIter = rIter;
|
||||
tmpInputKey = new SizedWritable<IK>(iKClass);
|
||||
tmpInputValue = new SizedWritable<IV>(iVClass);
|
||||
|
||||
if (null != iKClass && null != iVClass) {
|
||||
this.serializer = new KVSerializer<IK, IV>(iKClass, iVClass);
|
||||
}
|
||||
this.outputBuffer = target.getOutputBuffer();
|
||||
this.target = target;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int load() throws IOException {
|
||||
if (closed) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
if (null == outputBuffer) {
|
||||
throw new IOException("output buffer not set");
|
||||
}
|
||||
|
||||
this.nativeWriter = new ByteBufferDataWriter(target);
|
||||
outputBuffer.rewind();
|
||||
|
||||
int written = 0;
|
||||
boolean firstKV = true;
|
||||
|
||||
if (inputKVBufferd) {
|
||||
written += serializer.serializeKV(nativeWriter, tmpInputKey, tmpInputValue);
|
||||
inputKVBufferd = false;
|
||||
firstKV = false;
|
||||
}
|
||||
|
||||
while (rIter.next()) {
|
||||
inputKVBufferd = false;
|
||||
tmpInputKey.readFields(rIter.getKey());
|
||||
tmpInputValue.readFields(rIter.getValue());
|
||||
serializer.updateLength(tmpInputKey, tmpInputValue);
|
||||
|
||||
final int kvSize = tmpInputKey.length + tmpInputValue.length + KV_HEADER_LENGTH;
|
||||
|
||||
if (!firstKV && nativeWriter.shortOfSpace(kvSize)) {
|
||||
inputKVBufferd = true;
|
||||
break;
|
||||
} else {
|
||||
written += serializer.serializeKV(nativeWriter, tmpInputKey, tmpInputValue);
|
||||
firstKV = false;
|
||||
}
|
||||
}
|
||||
|
||||
if (nativeWriter.hasUnFlushedData()) {
|
||||
nativeWriter.flush();
|
||||
}
|
||||
return written;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
if (null != rIter) {
|
||||
rIter.close();
|
||||
}
|
||||
if (null != nativeWriter) {
|
||||
nativeWriter.close();
|
||||
}
|
||||
closed = true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,203 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.handlers;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.apache.hadoop.mapred.RawKeyValueIterator;
|
||||
import org.apache.hadoop.mapred.nativetask.Constants;
|
||||
import org.apache.hadoop.mapred.nativetask.DataReceiver;
|
||||
import org.apache.hadoop.mapred.nativetask.NativeDataSource;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.BufferType;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.ByteBufferDataReader;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.InputBuffer;
|
||||
import org.apache.hadoop.util.Progress;
|
||||
|
||||
/**
|
||||
* actively signal a {@link BufferPullee} to load data into buffer and receive
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class BufferPuller implements RawKeyValueIterator, DataReceiver {
|
||||
|
||||
private static Log LOG = LogFactory.getLog(BufferPuller.class);
|
||||
|
||||
public final static int KV_HEADER_LENGTH = Constants.SIZEOF_KV_LENGTH;
|
||||
|
||||
byte[] keyBytes = new byte[0];
|
||||
byte[] valueBytes = new byte[0];
|
||||
|
||||
private InputBuffer inputBuffer;
|
||||
private InputBuffer asideBuffer;
|
||||
|
||||
int remain = 0;
|
||||
|
||||
private ByteBufferDataReader nativeReader;
|
||||
|
||||
DataInputBuffer keyBuffer = new DataInputBuffer();
|
||||
DataInputBuffer valueBuffer = new DataInputBuffer();
|
||||
|
||||
private boolean noMoreData = false;
|
||||
|
||||
private NativeDataSource input;
|
||||
private boolean closed = false;
|
||||
|
||||
public BufferPuller(NativeDataSource handler) throws IOException {
|
||||
this.input = handler;
|
||||
this.inputBuffer = handler.getInputBuffer();
|
||||
nativeReader = new ByteBufferDataReader(null);
|
||||
this.asideBuffer = new InputBuffer(BufferType.HEAP_BUFFER, inputBuffer.capacity());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataInputBuffer getKey() throws IOException {
|
||||
return keyBuffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataInputBuffer getValue() throws IOException {
|
||||
return valueBuffer;
|
||||
}
|
||||
|
||||
public void reset() {
|
||||
noMoreData = false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean next() throws IOException {
|
||||
if (closed) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (noMoreData) {
|
||||
return false;
|
||||
}
|
||||
final int asideRemain = asideBuffer.remaining();
|
||||
final int inputRemain = inputBuffer.remaining();
|
||||
|
||||
if (asideRemain == 0 && inputRemain == 0) {
|
||||
input.loadData();
|
||||
}
|
||||
|
||||
if (asideBuffer.remaining() > 0) {
|
||||
return nextKeyValue(asideBuffer);
|
||||
} else if (inputBuffer.remaining() > 0) {
|
||||
return nextKeyValue(inputBuffer);
|
||||
} else {
|
||||
noMoreData = true;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private boolean nextKeyValue(InputBuffer buffer) throws IOException {
|
||||
if (closed) {
|
||||
return false;
|
||||
}
|
||||
|
||||
nativeReader.reset(buffer);
|
||||
|
||||
final int keyLength = nativeReader.readInt();
|
||||
if (keyBytes.length < keyLength) {
|
||||
keyBytes = new byte[keyLength];
|
||||
}
|
||||
|
||||
final int valueLength = nativeReader.readInt();
|
||||
if (valueBytes.length < valueLength) {
|
||||
valueBytes = new byte[valueLength];
|
||||
}
|
||||
|
||||
IOUtils.readFully(nativeReader, keyBytes, 0, keyLength);
|
||||
IOUtils.readFully(nativeReader, valueBytes, 0, valueLength);
|
||||
|
||||
keyBuffer.reset(keyBytes, keyLength);
|
||||
valueBuffer.reset(valueBytes, valueLength);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean receiveData() throws IOException {
|
||||
if (closed) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final ByteBuffer input = inputBuffer.getByteBuffer();
|
||||
|
||||
if (null != asideBuffer && asideBuffer.length() > 0) {
|
||||
if (asideBuffer.remaining() > 0) {
|
||||
final byte[] output = asideBuffer.getByteBuffer().array();
|
||||
final int write = Math.min(asideBuffer.remaining(), input.remaining());
|
||||
input.get(output, asideBuffer.position(), write);
|
||||
asideBuffer.position(asideBuffer.position() + write);
|
||||
}
|
||||
|
||||
if (asideBuffer.remaining() == 0) {
|
||||
asideBuffer.position(0);
|
||||
}
|
||||
}
|
||||
|
||||
if (input.remaining() == 0) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (input.remaining() < KV_HEADER_LENGTH) {
|
||||
throw new IOException("incomplete data, input length is: " + input.remaining());
|
||||
}
|
||||
final int position = input.position();
|
||||
final int keyLength = input.getInt();
|
||||
final int valueLength = input.getInt();
|
||||
input.position(position);
|
||||
final int kvLength = keyLength + valueLength + KV_HEADER_LENGTH;
|
||||
final int remaining = input.remaining();
|
||||
|
||||
if (kvLength > remaining) {
|
||||
if (null == asideBuffer || asideBuffer.capacity() < kvLength) {
|
||||
asideBuffer = new InputBuffer(BufferType.HEAP_BUFFER, kvLength);
|
||||
}
|
||||
asideBuffer.rewind(0, kvLength);
|
||||
|
||||
input.get(asideBuffer.array(), 0, remaining);
|
||||
asideBuffer.position(remaining);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Progress getProgress() {
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the iterator so that the underlying streams can be closed.
|
||||
*/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
if (null != nativeReader) {
|
||||
nativeReader.close();
|
||||
}
|
||||
closed = true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,151 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.handlers;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.mapred.RecordWriter;
|
||||
import org.apache.hadoop.mapred.nativetask.Constants;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.BufferType;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.ByteBufferDataReader;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.InputBuffer;
|
||||
import org.apache.hadoop.mapred.nativetask.serde.KVSerializer;
|
||||
import org.apache.hadoop.mapred.nativetask.util.SizedWritable;
|
||||
|
||||
/**
|
||||
* collect data when signaled
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class BufferPushee<OK, OV> implements Closeable {
|
||||
|
||||
private static Log LOG = LogFactory.getLog(BufferPushee.class);
|
||||
|
||||
public final static int KV_HEADER_LENGTH = Constants.SIZEOF_KV_LENGTH;
|
||||
|
||||
private InputBuffer asideBuffer;
|
||||
private final SizedWritable<OK> tmpOutputKey;
|
||||
private final SizedWritable<OV> tmpOutputValue;
|
||||
private RecordWriter<OK, OV> writer;
|
||||
private ByteBufferDataReader nativeReader;
|
||||
|
||||
private KVSerializer<OK, OV> deserializer;
|
||||
private boolean closed = false;
|
||||
|
||||
public BufferPushee(Class<OK> oKClass, Class<OV> oVClass,
|
||||
RecordWriter<OK, OV> writer) throws IOException {
|
||||
tmpOutputKey = new SizedWritable<OK>(oKClass);
|
||||
tmpOutputValue = new SizedWritable<OV>(oVClass);
|
||||
|
||||
this.writer = writer;
|
||||
|
||||
if (null != oKClass && null != oVClass) {
|
||||
this.deserializer = new KVSerializer<OK, OV>(oKClass, oVClass);
|
||||
}
|
||||
this.nativeReader = new ByteBufferDataReader(null);
|
||||
}
|
||||
|
||||
public boolean collect(InputBuffer buffer) throws IOException {
|
||||
if (closed) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final ByteBuffer input = buffer.getByteBuffer();
|
||||
if (null != asideBuffer && asideBuffer.length() > 0) {
|
||||
if (asideBuffer.remaining() > 0) {
|
||||
final byte[] output = asideBuffer.getByteBuffer().array();
|
||||
final int write = Math.min(asideBuffer.remaining(), input.remaining());
|
||||
input.get(output, asideBuffer.position(), write);
|
||||
asideBuffer.position(asideBuffer.position() + write);
|
||||
}
|
||||
|
||||
if (asideBuffer.remaining() == 0 && asideBuffer.position() > 0) {
|
||||
asideBuffer.position(0);
|
||||
write(asideBuffer);
|
||||
asideBuffer.rewind(0, 0);
|
||||
}
|
||||
}
|
||||
|
||||
if (input.remaining() == 0) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (input.remaining() < KV_HEADER_LENGTH) {
|
||||
throw new IOException("incomplete data, input length is: " + input.remaining());
|
||||
}
|
||||
final int position = input.position();
|
||||
final int keyLength = input.getInt();
|
||||
final int valueLength = input.getInt();
|
||||
input.position(position);
|
||||
final int kvLength = keyLength + valueLength + KV_HEADER_LENGTH;
|
||||
final int remaining = input.remaining();
|
||||
|
||||
if (kvLength > remaining) {
|
||||
if (null == asideBuffer || asideBuffer.capacity() < kvLength) {
|
||||
asideBuffer = new InputBuffer(BufferType.HEAP_BUFFER, kvLength);
|
||||
}
|
||||
asideBuffer.rewind(0, kvLength);
|
||||
|
||||
input.get(asideBuffer.array(), 0, remaining);
|
||||
asideBuffer.position(remaining);
|
||||
} else {
|
||||
write(buffer);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private boolean write(InputBuffer input) throws IOException {
|
||||
if (closed) {
|
||||
return false;
|
||||
}
|
||||
int totalRead = 0;
|
||||
final int remain = input.remaining();
|
||||
this.nativeReader.reset(input);
|
||||
while (remain > totalRead) {
|
||||
final int read = deserializer.deserializeKV(nativeReader, tmpOutputKey, tmpOutputValue);
|
||||
if (read != 0) {
|
||||
totalRead += read;
|
||||
writer.write((OK) (tmpOutputKey.v), (OV) (tmpOutputValue.v));
|
||||
}
|
||||
}
|
||||
if (remain != totalRead) {
|
||||
throw new IOException("We expect to read " + remain +
|
||||
", but we actually read: " + totalRead);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
if (null != writer) {
|
||||
writer.close(null);
|
||||
}
|
||||
if (null != nativeReader) {
|
||||
nativeReader.close();
|
||||
}
|
||||
closed = true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,91 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.handlers;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.mapred.OutputCollector;
|
||||
import org.apache.hadoop.mapred.nativetask.NativeDataTarget;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.ByteBufferDataWriter;
|
||||
import org.apache.hadoop.mapred.nativetask.serde.IKVSerializer;
|
||||
import org.apache.hadoop.mapred.nativetask.serde.KVSerializer;
|
||||
import org.apache.hadoop.mapred.nativetask.util.SizedWritable;
|
||||
|
||||
/**
|
||||
* actively push data into a buffer and signal a {@link BufferPushee} to collect it
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class BufferPusher<K, V> implements OutputCollector<K, V> {
|
||||
|
||||
private static Log LOG = LogFactory.getLog(BufferPusher.class);
|
||||
|
||||
private final SizedWritable<K> tmpInputKey;
|
||||
private final SizedWritable<V> tmpInputValue;
|
||||
private ByteBufferDataWriter out;
|
||||
IKVSerializer serializer;
|
||||
private boolean closed = false;
|
||||
|
||||
public BufferPusher(Class<K> iKClass, Class<V> iVClass,
|
||||
NativeDataTarget target) throws IOException {
|
||||
tmpInputKey = new SizedWritable<K>(iKClass);
|
||||
tmpInputValue = new SizedWritable<V>(iVClass);
|
||||
|
||||
if (null != iKClass && null != iVClass) {
|
||||
this.serializer = new KVSerializer<K, V>(iKClass, iVClass);
|
||||
}
|
||||
this.out = new ByteBufferDataWriter(target);
|
||||
}
|
||||
|
||||
public void collect(K key, V value, int partition) throws IOException {
|
||||
tmpInputKey.reset(key);
|
||||
tmpInputValue.reset(value);
|
||||
serializer.serializePartitionKV(out, partition, tmpInputKey, tmpInputValue);
|
||||
};
|
||||
|
||||
@Override
|
||||
public void collect(K key, V value) throws IOException {
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
tmpInputKey.reset(key);
|
||||
tmpInputValue.reset(value);
|
||||
serializer.serializeKV(out, tmpInputKey, tmpInputValue);
|
||||
};
|
||||
|
||||
public void flush() throws IOException {
|
||||
if (null != out) {
|
||||
if (out.hasUnFlushedData()) {
|
||||
out.flush();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
if (null != out) {
|
||||
out.close();
|
||||
}
|
||||
closed = true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,145 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.handlers;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.mapred.Counters.Counter;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.Task.CombinerRunner;
|
||||
import org.apache.hadoop.mapred.nativetask.Command;
|
||||
import org.apache.hadoop.mapred.nativetask.CommandDispatcher;
|
||||
import org.apache.hadoop.mapred.nativetask.Constants;
|
||||
import org.apache.hadoop.mapred.nativetask.DataChannel;
|
||||
import org.apache.hadoop.mapred.nativetask.ICombineHandler;
|
||||
import org.apache.hadoop.mapred.nativetask.INativeHandler;
|
||||
import org.apache.hadoop.mapred.nativetask.NativeBatchProcessor;
|
||||
import org.apache.hadoop.mapred.nativetask.TaskContext;
|
||||
import org.apache.hadoop.mapred.nativetask.serde.SerializationFramework;
|
||||
import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer;
|
||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.TaskCounter;
|
||||
|
||||
class CombinerHandler<K, V> implements ICombineHandler, CommandDispatcher {
|
||||
public static final String NAME = "NativeTask.CombineHandler";
|
||||
private static Log LOG = LogFactory.getLog(NativeCollectorOnlyHandler.class);
|
||||
public static final Command LOAD = new Command(1, "Load");
|
||||
public static final Command COMBINE = new Command(4, "Combine");
|
||||
public final CombinerRunner<K, V> combinerRunner;
|
||||
|
||||
private final INativeHandler nativeHandler;
|
||||
private final BufferPuller puller;
|
||||
private final BufferPusher<K, V> kvPusher;
|
||||
private boolean closed = false;
|
||||
|
||||
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()));
|
||||
String combinerClazz = conf.get(Constants.MAPRED_COMBINER_CLASS);
|
||||
if (null == combinerClazz) {
|
||||
combinerClazz = conf.get(MRJobConfig.COMBINE_CLASS_ATTR);
|
||||
}
|
||||
|
||||
if (null == combinerClazz) {
|
||||
return null;
|
||||
} else {
|
||||
LOG.info("NativeTask Combiner is enabled, class = " + combinerClazz);
|
||||
}
|
||||
|
||||
final Counter combineInputCounter = context.getTaskReporter().getCounter(
|
||||
TaskCounter.COMBINE_INPUT_RECORDS);
|
||||
|
||||
final CombinerRunner<K, V> combinerRunner = CombinerRunner.create(
|
||||
conf, context.getTaskAttemptId(),
|
||||
combineInputCounter, context.getTaskReporter(), null);
|
||||
|
||||
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(),
|
||||
nativeHandler);
|
||||
final BufferPuller puller = new BufferPuller(nativeHandler);
|
||||
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 {
|
||||
this.nativeHandler = nativeHandler;
|
||||
this.combinerRunner = combiner;
|
||||
this.puller = puller;
|
||||
this.kvPusher = kvPusher;
|
||||
nativeHandler.setCommandDispatcher(this);
|
||||
nativeHandler.setDataReceiver(puller);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadWriteBuffer onCall(Command command, ReadWriteBuffer parameter) throws IOException {
|
||||
if (null == command) {
|
||||
return null;
|
||||
}
|
||||
if (command.equals(COMBINE)) {
|
||||
combine();
|
||||
}
|
||||
return null;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void combine() throws IOException{
|
||||
try {
|
||||
puller.reset();
|
||||
combinerRunner.combine(puller, kvPusher);
|
||||
kvPusher.flush();
|
||||
return;
|
||||
} catch (Exception e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getId() {
|
||||
return nativeHandler.getNativeHandler();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (null != puller) {
|
||||
puller.close();
|
||||
}
|
||||
|
||||
if (null != kvPusher) {
|
||||
kvPusher.close();
|
||||
}
|
||||
|
||||
if (null != nativeHandler) {
|
||||
nativeHandler.close();
|
||||
}
|
||||
closed = true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,37 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.handlers;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* an IDataLoader loads data on demand
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface IDataLoader {
|
||||
|
||||
/**
|
||||
* @return size of data loaded
|
||||
*/
|
||||
public int load() throws IOException;
|
||||
|
||||
public void close() throws IOException;
|
||||
}
|
|
@ -0,0 +1,171 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.handlers;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.TaskAttemptID;
|
||||
import org.apache.hadoop.mapred.nativetask.Command;
|
||||
import org.apache.hadoop.mapred.nativetask.CommandDispatcher;
|
||||
import org.apache.hadoop.mapred.nativetask.DataChannel;
|
||||
import org.apache.hadoop.mapred.nativetask.ICombineHandler;
|
||||
import org.apache.hadoop.mapred.nativetask.INativeHandler;
|
||||
import org.apache.hadoop.mapred.nativetask.NativeBatchProcessor;
|
||||
import org.apache.hadoop.mapred.nativetask.TaskContext;
|
||||
import org.apache.hadoop.mapred.nativetask.util.NativeTaskOutput;
|
||||
import org.apache.hadoop.mapred.nativetask.util.OutputUtil;
|
||||
import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer;
|
||||
|
||||
/**
|
||||
* Java Record Reader + Java Mapper + Native Collector
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
@InterfaceAudience.Private
|
||||
public class NativeCollectorOnlyHandler<K, V> implements CommandDispatcher, Closeable {
|
||||
|
||||
public static final String NAME = "NativeTask.MCollectorOutputHandler";
|
||||
private static Log LOG = LogFactory.getLog(NativeCollectorOnlyHandler.class);
|
||||
public static final Command GET_OUTPUT_PATH =
|
||||
new Command(100, "GET_OUTPUT_PATH");
|
||||
public static final Command GET_OUTPUT_INDEX_PATH =
|
||||
new Command(101, "GET_OUTPUT_INDEX_PATH");
|
||||
public static final Command GET_SPILL_PATH =
|
||||
new Command(102, "GET_SPILL_PATH");
|
||||
public static final Command GET_COMBINE_HANDLER =
|
||||
new Command(103, "GET_COMBINE_HANDLER");
|
||||
|
||||
private NativeTaskOutput output;
|
||||
private int spillNumber = 0;
|
||||
private ICombineHandler combinerHandler = null;
|
||||
private final BufferPusher<K, V> kvPusher;
|
||||
private final INativeHandler nativeHandler;
|
||||
private boolean closed = false;
|
||||
|
||||
public static <K, V> NativeCollectorOnlyHandler<K, V> create(TaskContext context)
|
||||
throws IOException {
|
||||
|
||||
|
||||
ICombineHandler combinerHandler = null;
|
||||
try {
|
||||
final TaskContext combineContext = context.copyOf();
|
||||
combineContext.setInputKeyClass(context.getOutputKeyClass());
|
||||
combineContext.setInputValueClass(context.getOutputValueClass());
|
||||
|
||||
combinerHandler = CombinerHandler.create(combineContext);
|
||||
} catch (final ClassNotFoundException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
||||
if (null != combinerHandler) {
|
||||
LOG.info("[NativeCollectorOnlyHandler] combiner is not null");
|
||||
}
|
||||
|
||||
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(),
|
||||
nativeHandler);
|
||||
|
||||
return new NativeCollectorOnlyHandler<K, V>(context, nativeHandler, kvPusher, combinerHandler);
|
||||
}
|
||||
|
||||
protected NativeCollectorOnlyHandler(TaskContext context, INativeHandler nativeHandler,
|
||||
BufferPusher<K, V> kvPusher, ICombineHandler combiner) throws IOException {
|
||||
Configuration conf = context.getConf();
|
||||
TaskAttemptID id = context.getTaskAttemptId();
|
||||
if (null == id) {
|
||||
this.output = OutputUtil.createNativeTaskOutput(conf, "");
|
||||
} else {
|
||||
this.output = OutputUtil.createNativeTaskOutput(context.getConf(), context.getTaskAttemptId()
|
||||
.toString());
|
||||
}
|
||||
this.combinerHandler = combiner;
|
||||
this.kvPusher = kvPusher;
|
||||
this.nativeHandler = nativeHandler;
|
||||
nativeHandler.setCommandDispatcher(this);
|
||||
}
|
||||
|
||||
public void collect(K key, V value, int partition) throws IOException {
|
||||
kvPusher.collect(key, value, partition);
|
||||
};
|
||||
|
||||
public void flush() throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (null != kvPusher) {
|
||||
kvPusher.close();
|
||||
}
|
||||
|
||||
if (null != combinerHandler) {
|
||||
combinerHandler.close();
|
||||
}
|
||||
|
||||
if (null != nativeHandler) {
|
||||
nativeHandler.close();
|
||||
}
|
||||
closed = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadWriteBuffer onCall(Command command, ReadWriteBuffer parameter) throws IOException {
|
||||
Path p = null;
|
||||
if (null == command) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (command.equals(GET_OUTPUT_PATH)) {
|
||||
p = output.getOutputFileForWrite(-1);
|
||||
} else if (command.equals(GET_OUTPUT_INDEX_PATH)) {
|
||||
p = output.getOutputIndexFileForWrite(-1);
|
||||
} else if (command.equals(GET_SPILL_PATH)) {
|
||||
p = output.getSpillFileForWrite(spillNumber++, -1);
|
||||
|
||||
} else if (command.equals(GET_COMBINE_HANDLER)) {
|
||||
if (null == combinerHandler) {
|
||||
return null;
|
||||
}
|
||||
final ReadWriteBuffer result = new ReadWriteBuffer(8);
|
||||
|
||||
result.writeLong(combinerHandler.getId());
|
||||
return result;
|
||||
} else {
|
||||
throw new IOException("Illegal command: " + command.toString());
|
||||
}
|
||||
if (p != null) {
|
||||
final ReadWriteBuffer result = new ReadWriteBuffer();
|
||||
result.writeString(p.toUri().getPath());
|
||||
return result;
|
||||
} else {
|
||||
throw new IOException("MapOutputFile can't allocate spill/output file");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.nativetask.INativeComparable;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class BoolWritableSerializer extends DefaultSerializer implements
|
||||
INativeComparable {
|
||||
|
||||
@Override
|
||||
public int getLength(Writable w) throws IOException {
|
||||
return 1;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.nativetask.INativeComparable;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class ByteWritableSerializer extends DefaultSerializer implements
|
||||
INativeComparable {
|
||||
|
||||
@Override
|
||||
public int getLength(Writable w) throws IOException {
|
||||
return 1;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.mapred.nativetask.INativeComparable;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class BytesWritableSerializer
|
||||
implements INativeComparable, INativeSerializer<BytesWritable> {
|
||||
|
||||
@Override
|
||||
public int getLength(BytesWritable w) throws IOException {
|
||||
return w.getLength();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serialize(BytesWritable w, DataOutput out) throws IOException {
|
||||
out.write(w.getBytes(), 0, w.getLength());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deserialize(DataInput in, int length, BytesWritable w) throws IOException {
|
||||
w.setSize(length);
|
||||
in.readFully(w.getBytes(), 0, length);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,71 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public class DefaultSerializer implements INativeSerializer<Writable> {
|
||||
|
||||
static class ModifiedByteArrayOutputStream extends ByteArrayOutputStream {
|
||||
|
||||
public byte[] getBuffer() {
|
||||
return this.buf;
|
||||
}
|
||||
}
|
||||
|
||||
private final ModifiedByteArrayOutputStream outBuffer = new ModifiedByteArrayOutputStream();
|
||||
private final DataOutputStream outData = new DataOutputStream(outBuffer);
|
||||
private Writable buffered = null;
|
||||
private int bufferedLength = -1;
|
||||
|
||||
@Override
|
||||
public int getLength(Writable w) throws IOException {
|
||||
// if (w == buffered) {
|
||||
// return bufferedLength;
|
||||
// }
|
||||
buffered = null;
|
||||
bufferedLength = -1;
|
||||
|
||||
outBuffer.reset();
|
||||
w.write(outData);
|
||||
bufferedLength = outBuffer.size();
|
||||
buffered = w;
|
||||
return bufferedLength;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serialize(Writable w, DataOutput out) throws IOException {
|
||||
w.write(out);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deserialize(DataInput in, int length, Writable w) throws IOException {
|
||||
w.readFields(in);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.nativetask.INativeComparable;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class DoubleWritableSerializer extends DefaultSerializer implements
|
||||
INativeComparable {
|
||||
|
||||
@Override
|
||||
public int getLength(Writable w) throws IOException {
|
||||
return 8;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,36 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.nativetask.INativeComparable;
|
||||
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class FloatWritableSerializer extends DefaultSerializer implements
|
||||
INativeComparable {
|
||||
|
||||
@Override
|
||||
public int getLength(Writable w) throws IOException {
|
||||
return 4;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.DataInputStream;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.DataOutputStream;
|
||||
import org.apache.hadoop.mapred.nativetask.util.SizedWritable;
|
||||
|
||||
/**
|
||||
* serializes key-value pair
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface IKVSerializer {
|
||||
|
||||
/**
|
||||
* update the length field of SizedWritable
|
||||
*/
|
||||
public void updateLength(SizedWritable<?> key, SizedWritable<?> value) throws IOException;
|
||||
|
||||
public int serializeKV(DataOutputStream out, SizedWritable<?> key,
|
||||
SizedWritable<?> value) throws IOException;
|
||||
|
||||
public int serializePartitionKV(DataOutputStream out, int partitionId,
|
||||
SizedWritable<?> key, SizedWritable<?> value)
|
||||
throws IOException;
|
||||
|
||||
public int deserializeKV(DataInputStream in, SizedWritable<?> key, SizedWritable<?> value)
|
||||
throws IOException;
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* an INativeSerializer serializes and deserializes data transferred between
|
||||
* Java and native. {@link DefaultSerializer} provides default implementations.
|
||||
*
|
||||
* Note: if you implemented your customized NativeSerializer instead of DefaultSerializer,
|
||||
* you have to make sure the native side can serialize it correctly.
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public interface INativeSerializer<T> {
|
||||
|
||||
/**
|
||||
* get length of data to be serialized. If the data length is already known (like IntWritable)
|
||||
* and could immediately be returned from this method, it is good chance to implement customized
|
||||
* NativeSerializer for efficiency
|
||||
*/
|
||||
public int getLength(T w) throws IOException;
|
||||
|
||||
public void serialize(T w, DataOutput out) throws IOException;
|
||||
|
||||
public void deserialize(DataInput in, int length, T w) throws IOException;
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.nativetask.INativeComparable;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class IntWritableSerializer extends DefaultSerializer implements
|
||||
INativeComparable {
|
||||
|
||||
@Override
|
||||
public int getLength(Writable w) throws IOException {
|
||||
return 4;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,115 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.nativetask.Constants;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.DataInputStream;
|
||||
import org.apache.hadoop.mapred.nativetask.buffer.DataOutputStream;
|
||||
import org.apache.hadoop.mapred.nativetask.util.SizedWritable;
|
||||
|
||||
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class KVSerializer<K, V> implements IKVSerializer {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(KVSerializer.class);
|
||||
|
||||
public static final int KV_HEAD_LENGTH = Constants.SIZEOF_KV_LENGTH;
|
||||
|
||||
private final INativeSerializer<Writable> keySerializer;
|
||||
private final INativeSerializer<Writable> valueSerializer;
|
||||
|
||||
public KVSerializer(Class<K> kclass, Class<V> vclass) throws IOException {
|
||||
|
||||
this.keySerializer = NativeSerialization.getInstance().getSerializer(kclass);
|
||||
this.valueSerializer = NativeSerialization.getInstance().getSerializer(vclass);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateLength(SizedWritable<?> key, SizedWritable<?> value) throws IOException {
|
||||
key.length = keySerializer.getLength(key.v);
|
||||
value.length = valueSerializer.getLength(value.v);
|
||||
return;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int serializeKV(DataOutputStream out, SizedWritable<?> key, SizedWritable<?> value)
|
||||
throws IOException {
|
||||
return serializePartitionKV(out, -1, key, value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int serializePartitionKV(DataOutputStream out, int partitionId,
|
||||
SizedWritable<?> key, SizedWritable<?> value)
|
||||
throws IOException {
|
||||
|
||||
if (key.length == SizedWritable.INVALID_LENGTH ||
|
||||
value.length == SizedWritable.INVALID_LENGTH) {
|
||||
updateLength(key, value);
|
||||
}
|
||||
|
||||
final int keyLength = key.length;
|
||||
final int valueLength = value.length;
|
||||
|
||||
int bytesWritten = KV_HEAD_LENGTH + keyLength + valueLength;
|
||||
if (partitionId != -1) {
|
||||
bytesWritten += Constants.SIZEOF_PARTITION_LENGTH;
|
||||
}
|
||||
|
||||
if (out.hasUnFlushedData() && out.shortOfSpace(bytesWritten)) {
|
||||
out.flush();
|
||||
}
|
||||
|
||||
if (partitionId != -1) {
|
||||
out.writeInt(partitionId);
|
||||
}
|
||||
|
||||
out.writeInt(keyLength);
|
||||
out.writeInt(valueLength);
|
||||
|
||||
keySerializer.serialize(key.v, out);
|
||||
valueSerializer.serialize(value.v, out);
|
||||
|
||||
return bytesWritten;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int deserializeKV(DataInputStream in, SizedWritable<?> key,
|
||||
SizedWritable<?> value) throws IOException {
|
||||
|
||||
if (!in.hasUnReadData()) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
key.length = in.readInt();
|
||||
value.length = in.readInt();
|
||||
|
||||
keySerializer.deserialize(in, key.length, key.v);
|
||||
valueSerializer.deserialize(in, value.length, value.v);
|
||||
|
||||
return key.length + value.length + KV_HEAD_LENGTH;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,34 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.nativetask.INativeComparable;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class LongWritableSerializer extends DefaultSerializer implements
|
||||
INativeComparable {
|
||||
@Override
|
||||
public int getLength(Writable w) throws IOException {
|
||||
return 8;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,91 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class NativeSerialization {
|
||||
|
||||
private final ConcurrentHashMap<String, Class<?>> map =
|
||||
new ConcurrentHashMap<String, Class<?>>();
|
||||
|
||||
public boolean accept(Class<?> c) {
|
||||
return Writable.class.isAssignableFrom(c);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public INativeSerializer<Writable> getSerializer(Class<?> c) throws IOException {
|
||||
|
||||
if (null == c) {
|
||||
return null;
|
||||
}
|
||||
if (!Writable.class.isAssignableFrom(c)) {
|
||||
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);
|
||||
|
||||
if (null != serializer) {
|
||||
try {
|
||||
return (INativeSerializer<Writable>) serializer.newInstance();
|
||||
} catch (final Exception e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
return new DefaultSerializer();
|
||||
}
|
||||
|
||||
public void register(String klass, Class<?> serializer) throws IOException {
|
||||
if (null == klass || null == serializer) {
|
||||
throw new IOException("invalid arguments, klass or serializer is null");
|
||||
}
|
||||
|
||||
if (!INativeSerializer.class.isAssignableFrom(serializer)) {
|
||||
throw new IOException("Serializer is not assigable from INativeSerializer");
|
||||
}
|
||||
|
||||
final Class<?> storedSerializer = map.get(klass);
|
||||
if (null == storedSerializer) {
|
||||
map.put(klass, serializer);
|
||||
return;
|
||||
} else {
|
||||
if (!storedSerializer.getName().equals(serializer.getName())) {
|
||||
throw new IOException("Error! Serializer already registered, existing: " +
|
||||
storedSerializer.getName() + ", new: " +
|
||||
serializer.getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void reset() {
|
||||
map.clear();
|
||||
}
|
||||
|
||||
private static NativeSerialization instance = new NativeSerialization();
|
||||
|
||||
public static NativeSerialization getInstance() {
|
||||
return instance;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.nativetask.INativeComparable;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class NullWritableSerializer extends DefaultSerializer implements
|
||||
INativeComparable {
|
||||
|
||||
@Override
|
||||
public int getLength(Writable w) throws IOException {
|
||||
return 0;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public enum SerializationFramework {
|
||||
WRITABLE_SERIALIZATION(0), NATIVE_SERIALIZATION(1);
|
||||
|
||||
private int type;
|
||||
|
||||
SerializationFramework(int type) {
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
public int getType() {
|
||||
return type;
|
||||
}
|
||||
};
|
|
@ -0,0 +1,49 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapred.nativetask.INativeComparable;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class TextSerializer implements INativeSerializer<Text>, INativeComparable {
|
||||
|
||||
public TextSerializer() throws SecurityException, NoSuchMethodException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getLength(Text w) throws IOException {
|
||||
return w.getLength();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serialize(Text w, DataOutput out) throws IOException {
|
||||
out.write(w.getBytes(), 0, w.getLength());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deserialize(DataInput in, int length, Text w) throws IOException {
|
||||
w.readWithKnownLength(in, length);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,27 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.mapred.nativetask.INativeComparable;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class VIntWritableSerializer extends DefaultSerializer implements
|
||||
INativeComparable {
|
||||
}
|
|
@ -0,0 +1,27 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.serde;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.mapred.nativetask.INativeComparable;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class VLongWritableSerializer extends DefaultSerializer implements
|
||||
INativeComparable {
|
||||
}
|
|
@ -0,0 +1,172 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.util;
|
||||
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.common.primitives.Longs;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class BytesUtil {
|
||||
|
||||
private static final char[] HEX_CHARS =
|
||||
"0123456789abcdef".toCharArray();
|
||||
|
||||
/**
|
||||
* Converts a big-endian byte array to a long value.
|
||||
*
|
||||
* @param bytes array of bytes
|
||||
* @param offset offset into array
|
||||
*/
|
||||
public static long toLong(byte[] bytes, int offset) {
|
||||
return Longs.fromBytes(bytes[offset],
|
||||
bytes[offset + 1],
|
||||
bytes[offset + 2],
|
||||
bytes[offset + 3],
|
||||
bytes[offset + 4],
|
||||
bytes[offset + 5],
|
||||
bytes[offset + 6],
|
||||
bytes[offset + 7]);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a big-endian integer from a byte array to a primitive value.
|
||||
* @param bytes the array to parse from
|
||||
* @param offset the offset in the array
|
||||
*/
|
||||
public static int toInt(byte[] bytes, int offset) {
|
||||
return Ints.fromBytes(bytes[offset],
|
||||
bytes[offset + 1],
|
||||
bytes[offset + 2],
|
||||
bytes[offset + 3]);
|
||||
}
|
||||
|
||||
/**
|
||||
* Presumes float encoded as IEEE 754 floating-point "single format"
|
||||
* @param bytes byte array
|
||||
* @return Float made from passed byte array.
|
||||
*/
|
||||
public static float toFloat(byte [] bytes) {
|
||||
return toFloat(bytes, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Presumes float encoded as IEEE 754 floating-point "single format"
|
||||
* @param bytes array to convert
|
||||
* @param offset offset into array
|
||||
* @return Float made from passed byte array.
|
||||
*/
|
||||
public static float toFloat(byte [] bytes, int offset) {
|
||||
return Float.intBitsToFloat(toInt(bytes, offset));
|
||||
}
|
||||
|
||||
/**
|
||||
* @param bytes byte array
|
||||
* @return Return double made from passed bytes.
|
||||
*/
|
||||
public static double toDouble(final byte [] bytes) {
|
||||
return toDouble(bytes, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param bytes byte array
|
||||
* @param offset offset where double is
|
||||
* @return Return double made from passed bytes.
|
||||
*/
|
||||
public static double toDouble(final byte [] bytes, final int offset) {
|
||||
return Double.longBitsToDouble(toLong(bytes, offset));
|
||||
}
|
||||
|
||||
/**
|
||||
* Write a printable representation of a byte array.
|
||||
*
|
||||
* @param b byte array
|
||||
* @return the printable presentation
|
||||
* @see #toStringBinary(byte[], int, int)
|
||||
*/
|
||||
public static String toStringBinary(final byte [] b) {
|
||||
if (b == null)
|
||||
return "null";
|
||||
return toStringBinary(b, 0, b.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Write a printable representation of a byte array. Non-printable
|
||||
* characters are hex escaped in the format \\x%02X, eg:
|
||||
* \x00 \x05 etc
|
||||
*
|
||||
* @param b array to write out
|
||||
* @param off offset to start at
|
||||
* @param len length to write
|
||||
* @return string output
|
||||
*/
|
||||
public static String toStringBinary(final byte [] b, int off, int len) {
|
||||
StringBuilder result = new StringBuilder();
|
||||
// Just in case we are passed a 'len' that is > buffer length...
|
||||
if (off >= b.length) return result.toString();
|
||||
if (off + len > b.length) len = b.length - off;
|
||||
for (int i = off; i < off + len ; ++i ) {
|
||||
int ch = b[i] & 0xFF;
|
||||
if ( (ch >= '0' && ch <= '9')
|
||||
|| (ch >= 'A' && ch <= 'Z')
|
||||
|| (ch >= 'a' && ch <= 'z')
|
||||
|| " `~!@#$%^&*()-_=+[]{}|;:'\",.<>/?".indexOf(ch) >= 0 ) {
|
||||
result.append((char)ch);
|
||||
} else {
|
||||
result.append("\\x");
|
||||
result.append(HEX_CHARS[(ch >> 4) & 0x0F]);
|
||||
result.append(HEX_CHARS[ch & 0x0F]);
|
||||
}
|
||||
}
|
||||
return result.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a boolean to a byte array. True becomes -1
|
||||
* and false becomes 0.
|
||||
*
|
||||
* @param b value
|
||||
* @return <code>b</code> encoded in a byte array.
|
||||
*/
|
||||
public static byte [] toBytes(final boolean b) {
|
||||
return new byte[] { b ? (byte) -1 : (byte) 0 };
|
||||
}
|
||||
|
||||
/**
|
||||
* @param f float value
|
||||
* @return the float represented as byte []
|
||||
*/
|
||||
public static byte [] toBytes(final float f) {
|
||||
// Encode it as int
|
||||
return Ints.toByteArray(Float.floatToRawIntBits(f));
|
||||
}
|
||||
|
||||
/**
|
||||
* Serialize a double as the IEEE 754 double format output. The resultant
|
||||
* array will be 8 bytes long.
|
||||
*
|
||||
* @param d value
|
||||
* @return the double represented as byte []
|
||||
*/
|
||||
public static byte [] toBytes(final double d) {
|
||||
// Encode it as a long
|
||||
return Longs.toByteArray(Double.doubleToRawLongBits(d));
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.util;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public abstract class ConfigUtil {
|
||||
public static byte[][] toBytes(Configuration conf) {
|
||||
List<byte[]> nativeConfigs = new ArrayList<byte[]>();
|
||||
for (Map.Entry<String, String> e : conf) {
|
||||
nativeConfigs.add(e.getKey().getBytes(Charsets.UTF_8));
|
||||
nativeConfigs.add(e.getValue().getBytes(Charsets.UTF_8));
|
||||
}
|
||||
return nativeConfigs.toArray(new byte[nativeConfigs.size()][]);
|
||||
}
|
||||
|
||||
public static String booleansToString(boolean[] value) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
for (boolean b: value) {
|
||||
sb.append(b ? 1 : 0);
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,159 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.util;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.LocalDirAllocator;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.TaskID;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class LocalJobOutputFiles implements NativeTaskOutput {
|
||||
|
||||
static final String TASKTRACKER_OUTPUT = "output";
|
||||
static final String REDUCE_INPUT_FILE_FORMAT_STRING = "%s/map_%d.out";
|
||||
static final String SPILL_FILE_FORMAT_STRING = "%s/spill%d.out";
|
||||
static final String SPILL_INDEX_FILE_FORMAT_STRING = "%s/spill%d.out.index";
|
||||
static final String OUTPUT_FILE_FORMAT_STRING = "%s/file.out";
|
||||
static final String OUTPUT_FILE_INDEX_FORMAT_STRING = "%s/file.out.index";
|
||||
|
||||
private JobConf conf;
|
||||
private LocalDirAllocator lDirAlloc = new LocalDirAllocator("mapred.local.dir");
|
||||
|
||||
public LocalJobOutputFiles(Configuration conf, String id) {
|
||||
this.conf = new JobConf(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the path to local map output file created earlier
|
||||
*/
|
||||
public Path getOutputFile() throws IOException {
|
||||
String path = String.format(OUTPUT_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT);
|
||||
return lDirAlloc.getLocalPathToRead(path, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a local map output file name.
|
||||
*
|
||||
* @param size the size of the file
|
||||
*/
|
||||
public Path getOutputFileForWrite(long size) throws IOException {
|
||||
String path = String.format(OUTPUT_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT);
|
||||
return lDirAlloc.getLocalPathForWrite(path, size, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the path to a local map output index file created earlier
|
||||
*/
|
||||
public Path getOutputIndexFile() throws IOException {
|
||||
String path = String.format(OUTPUT_FILE_INDEX_FORMAT_STRING, TASKTRACKER_OUTPUT);
|
||||
return lDirAlloc.getLocalPathToRead(path, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a local map output index file name.
|
||||
*
|
||||
* @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);
|
||||
return lDirAlloc.getLocalPathForWrite(path, size, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a local map spill file created earlier.
|
||||
*
|
||||
* @param spillNumber the number
|
||||
*/
|
||||
public Path getSpillFile(int spillNumber) throws IOException {
|
||||
String path = String.format(SPILL_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, spillNumber);
|
||||
return lDirAlloc.getLocalPathToRead(path, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a local map spill file name.
|
||||
*
|
||||
* @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);
|
||||
return lDirAlloc.getLocalPathForWrite(path, size, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a local map spill index file created earlier
|
||||
*
|
||||
* @param spillNumber the number
|
||||
*/
|
||||
public Path getSpillIndexFile(int spillNumber) throws IOException {
|
||||
String path = String
|
||||
.format(SPILL_INDEX_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, spillNumber);
|
||||
return lDirAlloc.getLocalPathToRead(path, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a local map spill index file name.
|
||||
*
|
||||
* @param spillNumber the number
|
||||
* @param size the size of the file
|
||||
*/
|
||||
public Path getSpillIndexFileForWrite(int spillNumber, long size) throws IOException {
|
||||
String path = String
|
||||
.format(SPILL_INDEX_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, spillNumber);
|
||||
return lDirAlloc.getLocalPathForWrite(path, size, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a local reduce input file created earlier
|
||||
*
|
||||
* @param mapId a map task id
|
||||
*/
|
||||
public Path getInputFile(int mapId) throws IOException {
|
||||
return lDirAlloc.getLocalPathToRead(
|
||||
String.format(REDUCE_INPUT_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, Integer.valueOf(mapId)),
|
||||
conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a local reduce input file name.
|
||||
*
|
||||
* @param mapId a map task id
|
||||
* @param size the size of the file
|
||||
*/
|
||||
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);
|
||||
}
|
||||
|
||||
/** Removes all of the files related to a task. */
|
||||
public void removeAll() throws IOException {
|
||||
conf.deleteLocalFiles(TASKTRACKER_OUTPUT);
|
||||
}
|
||||
|
||||
public String getOutputName(int partition) {
|
||||
return String.format("part-%05d", partition);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,106 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.util;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.TaskID;
|
||||
|
||||
/**
|
||||
* base class of output files manager.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface NativeTaskOutput {
|
||||
|
||||
/**
|
||||
* Return the path to local map output file created earlier
|
||||
*/
|
||||
public Path getOutputFile() throws IOException;
|
||||
|
||||
/**
|
||||
* Create a local map output file name.
|
||||
*
|
||||
* @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
|
||||
*/
|
||||
public Path getOutputIndexFile() throws IOException;
|
||||
|
||||
/**
|
||||
* Create a local map output index file name.
|
||||
*
|
||||
* @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
|
||||
*/
|
||||
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
|
||||
*/
|
||||
public Path getSpillFileForWrite(int spillNumber, long size) throws IOException;
|
||||
|
||||
/**
|
||||
* Return a local map spill index file created earlier
|
||||
*
|
||||
* @param spillNumber the number
|
||||
*/
|
||||
public Path getSpillIndexFile(int spillNumber) throws IOException;
|
||||
|
||||
/**
|
||||
* Create a local map spill index file name.
|
||||
*
|
||||
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
|
||||
*/
|
||||
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
|
||||
*/
|
||||
public Path getInputFileForWrite(TaskID mapId, long size, Configuration conf) throws IOException;
|
||||
|
||||
/** Removes all of the files related to a task. */
|
||||
public void removeAll() throws IOException;
|
||||
|
||||
public String getOutputName(int partition);
|
||||
}
|
|
@ -0,0 +1,170 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.util;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.LocalDirAllocator;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
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.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class NativeTaskOutputFiles implements NativeTaskOutput {
|
||||
|
||||
static final String TASKTRACKER_OUTPUT = "output";
|
||||
static final String REDUCE_INPUT_FILE_FORMAT_STRING = "%s/map_%d.out";
|
||||
static final String SPILL_FILE_FORMAT_STRING = "%s/%s/spill%d.out";
|
||||
static final String SPILL_INDEX_FILE_FORMAT_STRING = "%s/%s/spill%d.out.index";
|
||||
static final String OUTPUT_FILE_FORMAT_STRING = "%s/%s/file.out";
|
||||
static final String OUTPUT_FILE_INDEX_FORMAT_STRING = "%s/%s/file.out.index";
|
||||
|
||||
private String id;
|
||||
private JobConf conf;
|
||||
private LocalDirAllocator lDirAlloc = new LocalDirAllocator("mapred.local.dir");
|
||||
|
||||
public NativeTaskOutputFiles(Configuration conf, String id) {
|
||||
this.conf = new JobConf(conf);
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the path to local map output file created earlier
|
||||
*/
|
||||
public Path getOutputFile() throws IOException {
|
||||
String path = String.format(OUTPUT_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, id);
|
||||
return lDirAlloc.getLocalPathToRead(path, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a local map output file name.
|
||||
*
|
||||
* @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);
|
||||
return lDirAlloc.getLocalPathForWrite(path, size, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the path to a local map output index file created earlier
|
||||
*/
|
||||
public Path getOutputIndexFile() throws IOException {
|
||||
String path = String.format(OUTPUT_FILE_INDEX_FORMAT_STRING, TASKTRACKER_OUTPUT, id);
|
||||
return lDirAlloc.getLocalPathToRead(path, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a local map output index file name.
|
||||
*
|
||||
* @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);
|
||||
return lDirAlloc.getLocalPathForWrite(path, size, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a local map spill file created earlier.
|
||||
*
|
||||
* @param spillNumber the number
|
||||
*/
|
||||
public Path getSpillFile(int spillNumber) throws IOException {
|
||||
String path = String.format(SPILL_FILE_FORMAT_STRING, id, TASKTRACKER_OUTPUT, spillNumber);
|
||||
return lDirAlloc.getLocalPathToRead(path, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a local map spill file name.
|
||||
*
|
||||
* @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);
|
||||
return lDirAlloc.getLocalPathForWrite(path, size, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a local map spill index file created earlier
|
||||
*
|
||||
* @param spillNumber the number
|
||||
*/
|
||||
public Path getSpillIndexFile(int spillNumber) throws IOException {
|
||||
String path = String
|
||||
.format(SPILL_INDEX_FILE_FORMAT_STRING, id, TASKTRACKER_OUTPUT, spillNumber);
|
||||
return lDirAlloc.getLocalPathToRead(path, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a local map spill index file name.
|
||||
*
|
||||
* @param spillNumber the number
|
||||
* @param size the size of the file
|
||||
*/
|
||||
public Path getSpillIndexFileForWrite(int spillNumber, long size) throws IOException {
|
||||
String path = String
|
||||
.format(SPILL_INDEX_FILE_FORMAT_STRING, id, TASKTRACKER_OUTPUT, spillNumber);
|
||||
return lDirAlloc.getLocalPathForWrite(path, size, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a local reduce input file created earlier
|
||||
*
|
||||
* @param mapId a map task id
|
||||
*/
|
||||
public Path getInputFile(int mapId) throws IOException {
|
||||
return lDirAlloc.getLocalPathToRead(
|
||||
String.format(REDUCE_INPUT_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, Integer.valueOf(mapId)),
|
||||
conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a local reduce input file name.
|
||||
*
|
||||
* @param mapId a map task id
|
||||
* @param size the size of the file
|
||||
*/
|
||||
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);
|
||||
}
|
||||
|
||||
/** Removes all of the files related to a task. */
|
||||
public void removeAll() throws IOException {
|
||||
conf.deleteLocalFiles(TASKTRACKER_OUTPUT);
|
||||
}
|
||||
|
||||
public String getOutputName(int partition) {
|
||||
return String.format("part-%05d", partition);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapred.nativetask.util;
|
||||
|
||||
import java.lang.reflect.Constructor;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class OutputUtil {
|
||||
|
||||
private static Log LOG = LogFactory.getLog(OutputUtil.class);
|
||||
public static final String NATIVE_TASK_OUTPUT_MANAGER = "nativetask.output.manager";
|
||||
|
||||
public static NativeTaskOutput createNativeTaskOutput(Configuration conf, String id) {
|
||||
Class<?> clazz = conf.getClass(OutputUtil.NATIVE_TASK_OUTPUT_MANAGER,
|
||||
NativeTaskOutputFiles.class);
|
||||
LOG.info(OutputUtil.NATIVE_TASK_OUTPUT_MANAGER + " = " + clazz.getName());
|
||||
try {
|
||||
Constructor<?> ctor = clazz.getConstructor(Configuration.class, String.class);
|
||||
ctor.setAccessible(true);
|
||||
NativeTaskOutput instance = (NativeTaskOutput) ctor.newInstance(conf, id);
|
||||
return instance;
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,159 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.util;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class ReadWriteBuffer {
|
||||
private byte[] _buff;
|
||||
private int _writePoint;
|
||||
private int _readPoint;
|
||||
final static int CACHE_LINE_SIZE = 16;
|
||||
|
||||
public ReadWriteBuffer(int length) {
|
||||
if (length > 0) {
|
||||
_buff = new byte[length];
|
||||
}
|
||||
}
|
||||
|
||||
public ReadWriteBuffer() {
|
||||
_buff = new byte[CACHE_LINE_SIZE];
|
||||
}
|
||||
|
||||
public ReadWriteBuffer(byte[] bytes) {
|
||||
_buff = bytes;
|
||||
_writePoint = 0;
|
||||
_readPoint = 0;
|
||||
}
|
||||
|
||||
public void reset(byte[] newBuff) {
|
||||
_buff = newBuff;
|
||||
_writePoint = 0;
|
||||
_readPoint = 0;
|
||||
}
|
||||
|
||||
public void setReadPoint(int pos) {
|
||||
_readPoint = pos;
|
||||
}
|
||||
|
||||
public void setWritePoint(int pos) {
|
||||
_writePoint = pos;
|
||||
}
|
||||
|
||||
public byte[] getBuff() {
|
||||
return _buff;
|
||||
}
|
||||
|
||||
public int getWritePoint() {
|
||||
return _writePoint;
|
||||
}
|
||||
|
||||
public int getReadPoint() {
|
||||
return _readPoint;
|
||||
}
|
||||
|
||||
public void writeInt(int v) {
|
||||
checkWriteSpaceAndResizeIfNecessary(4);
|
||||
|
||||
_buff[_writePoint + 0] = (byte) ((v >>> 0) & 0xFF);
|
||||
_buff[_writePoint + 1] = (byte) ((v >>> 8) & 0xFF);
|
||||
_buff[_writePoint + 2] = (byte) ((v >>> 16) & 0xFF);
|
||||
_buff[_writePoint + 3] = (byte) ((v >>> 24) & 0xFF);
|
||||
|
||||
_writePoint += 4;
|
||||
}
|
||||
|
||||
public void writeLong(long v) {
|
||||
checkWriteSpaceAndResizeIfNecessary(8);
|
||||
|
||||
_buff[_writePoint + 0] = (byte) (v >>> 0);
|
||||
_buff[_writePoint + 1] = (byte) (v >>> 8);
|
||||
_buff[_writePoint + 2] = (byte) (v >>> 16);
|
||||
_buff[_writePoint + 3] = (byte) (v >>> 24);
|
||||
_buff[_writePoint + 4] = (byte) (v >>> 32);
|
||||
_buff[_writePoint + 5] = (byte) (v >>> 40);
|
||||
_buff[_writePoint + 6] = (byte) (v >>> 48);
|
||||
_buff[_writePoint + 7] = (byte) (v >>> 56);
|
||||
|
||||
_writePoint += 8;
|
||||
}
|
||||
|
||||
public void writeBytes(byte b[], int off, int len) {
|
||||
writeInt(len);
|
||||
checkWriteSpaceAndResizeIfNecessary(len);
|
||||
System.arraycopy(b, off, _buff, _writePoint, len);
|
||||
_writePoint += len;
|
||||
}
|
||||
|
||||
public int readInt() {
|
||||
final int ch4 = 0xff & (_buff[_readPoint + 0]);
|
||||
final int ch3 = 0xff & (_buff[_readPoint + 1]);
|
||||
final int ch2 = 0xff & (_buff[_readPoint + 2]);
|
||||
final int ch1 = 0xff & (_buff[_readPoint + 3]);
|
||||
_readPoint += 4;
|
||||
return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0));
|
||||
}
|
||||
|
||||
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));
|
||||
|
||||
_readPoint += 8;
|
||||
return result;
|
||||
}
|
||||
|
||||
public byte[] readBytes() {
|
||||
final int length = readInt();
|
||||
final byte[] result = new byte[length];
|
||||
System.arraycopy(_buff, _readPoint, result, 0, length);
|
||||
_readPoint += length;
|
||||
return result;
|
||||
}
|
||||
|
||||
public void writeString(String str) {
|
||||
final byte[] bytes = str.getBytes(Charsets.UTF_8);
|
||||
writeBytes(bytes, 0, bytes.length);
|
||||
}
|
||||
|
||||
public String readString() {
|
||||
final byte[] bytes = readBytes();
|
||||
return new String(bytes, Charsets.UTF_8);
|
||||
}
|
||||
|
||||
private void checkWriteSpaceAndResizeIfNecessary(int toBeWritten) {
|
||||
|
||||
if (_buff.length - _writePoint >= toBeWritten) {
|
||||
return;
|
||||
}
|
||||
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,0 +1,55 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.hadoop.mapred.nativetask.util;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class SizedWritable<T> {
|
||||
public static final int INVALID_LENGTH = -1;
|
||||
|
||||
public int length = INVALID_LENGTH;
|
||||
public Writable v;
|
||||
|
||||
public SizedWritable(Class<?> klass) {
|
||||
if (null != klass) {
|
||||
v = (Writable) ReflectionUtils.newInstance(klass, null);
|
||||
}
|
||||
length = INVALID_LENGTH;
|
||||
}
|
||||
|
||||
public void readFields(DataInputBuffer key) throws IOException {
|
||||
if (null != key) {
|
||||
this.v.readFields(key);
|
||||
this.length = INVALID_LENGTH;
|
||||
} else {
|
||||
throw new IOException("input key is null");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void reset(T w) {
|
||||
this.v = (Writable) w;
|
||||
this.length = INVALID_LENGTH;
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,39 @@
|
|||
// Copyright 2006, Google Inc.
|
||||
// All rights reserved.
|
||||
//
|
||||
// Redistribution and use in source and binary forms, with or without
|
||||
// modification, are permitted provided that the following conditions are
|
||||
// met:
|
||||
//
|
||||
// * Redistributions of source code must retain the above copyright
|
||||
// notice, this list of conditions and the following disclaimer.
|
||||
// * Redistributions in binary form must reproduce the above
|
||||
// copyright notice, this list of conditions and the following disclaimer
|
||||
// in the documentation and/or other materials provided with the
|
||||
// distribution.
|
||||
// * Neither the name of Google Inc. nor the names of its
|
||||
// contributors may be used to endorse or promote products derived from
|
||||
// this software without specific prior written permission.
|
||||
//
|
||||
// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
|
||||
// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
|
||||
// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
|
||||
// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
|
||||
#include <iostream>
|
||||
|
||||
#include "gtest/gtest.h"
|
||||
|
||||
GTEST_API_ int main(int argc, char **argv) {
|
||||
std::cout << "Running main() from gtest_main.cc\n";
|
||||
|
||||
testing::InitGoogleTest(&argc, argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,507 @@
|
|||
/*
|
||||
* 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 NATIVETASK_H_
|
||||
#define NATIVETASK_H_
|
||||
|
||||
#include "lib/jniutils.h"
|
||||
#include <stdint.h>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include <map>
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
using std::string;
|
||||
using std::vector;
|
||||
using std::map;
|
||||
using std::pair;
|
||||
|
||||
/**
|
||||
* NativeObjectType
|
||||
*/
|
||||
enum NativeObjectType {
|
||||
UnknownObjectType = 0,
|
||||
BatchHandlerType = 1,
|
||||
};
|
||||
|
||||
/**
|
||||
* Enduim setting
|
||||
*
|
||||
*/
|
||||
enum Endium {
|
||||
LITTLE_ENDIUM = 0,
|
||||
LARGE_ENDIUM = 1
|
||||
};
|
||||
|
||||
#define NATIVE_COMBINER "native.combiner.class"
|
||||
#define NATIVE_PARTITIONER "native.partitioner.class"
|
||||
#define NATIVE_MAPPER "native.mapper.class"
|
||||
#define NATIVE_RECORDREADER "native.recordreader.class"
|
||||
#define NATIVE_RECORDWRITER "native.recordwriter.class"
|
||||
|
||||
#define NATIVE_REDUCER "native.reducer.class"
|
||||
#define NATIVE_HADOOP_VERSION "native.hadoop.version"
|
||||
|
||||
#define NATIVE_INPUT_SPLIT "native.input.split"
|
||||
#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 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"
|
||||
#define TOTAL_ORDER_PARTITIONER_MAX_TRIE_DEPTH "total.order.partitioner.max.trie.depth"
|
||||
#define FS_DEFAULT_NAME "fs.default.name"
|
||||
#define FS_DEFAULT_FS "fs.defaultFS"
|
||||
|
||||
#define NATIVE_SORT_TYPE "native.sort.type"
|
||||
#define MAPRED_SORT_AVOID "mapreduce.sort.avoidance"
|
||||
#define NATIVE_SORT_MAX_BLOCK_SIZE "native.sort.blocksize.max"
|
||||
#define MAPRED_COMPRESS_MAP_OUTPUT "mapreduce.map.output.compress"
|
||||
#define MAPRED_MAP_OUTPUT_COMPRESSION_CODEC "mapreduce.map.output.compress.codec"
|
||||
#define MAPRED_MAPOUTPUT_KEY_CLASS "mapreduce.map.output.key.class"
|
||||
#define MAPRED_OUTPUT_KEY_CLASS "mapreduce.job.output.key.class"
|
||||
#define MAPRED_MAPOUTPUT_VALUE_CLASS "mapreduce.map.output.value.class"
|
||||
#define MAPRED_OUTPUT_VALUE_CLASS "mapreduce.job.output.value.class"
|
||||
#define MAPRED_IO_SORT_MB "mapreduce.task.io.sort.mb"
|
||||
#define MAPRED_NUM_REDUCES "mapreduce.job.reduces"
|
||||
#define MAPRED_COMBINE_CLASS_OLD "mapred.combiner.class"
|
||||
#define MAPRED_COMBINE_CLASS_NEW "mapreduce.job.combine.class"
|
||||
|
||||
#define NATIVE_LOG_DEVICE "native.log.device"
|
||||
|
||||
//format: name=path,name=path,name=path
|
||||
#define NATIVE_CLASS_LIBRARY_BUILDIN "native.class.library.buildin"
|
||||
|
||||
#define NATIVE_MAPOUT_KEY_COMPARATOR "native.map.output.key.comparator"
|
||||
|
||||
extern const std::string NativeObjectTypeToString(NativeObjectType type);
|
||||
extern NativeObjectType NativeObjectTypeFromString(const std::string type);
|
||||
|
||||
/**
|
||||
* Objects that can be loaded dynamically from shared library,
|
||||
* and managed by NativeObjectFactory
|
||||
*/
|
||||
class NativeObject {
|
||||
public:
|
||||
virtual NativeObjectType type() {
|
||||
return UnknownObjectType;
|
||||
}
|
||||
|
||||
virtual ~NativeObject() {
|
||||
}
|
||||
;
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
NativeObject * ObjectCreator() {
|
||||
return new T();
|
||||
}
|
||||
|
||||
typedef NativeObject * (*ObjectCreatorFunc)();
|
||||
|
||||
typedef ObjectCreatorFunc (*GetObjectCreatorFunc)(const std::string & name);
|
||||
|
||||
typedef void * (*FunctionGetter)(const std::string & name);
|
||||
|
||||
typedef int32_t (*InitLibraryFunc)();
|
||||
|
||||
/**
|
||||
* Exceptions
|
||||
*/
|
||||
class HadoopException : public std::exception {
|
||||
private:
|
||||
std::string _reason;
|
||||
public:
|
||||
HadoopException(const string & what);
|
||||
virtual ~HadoopException() throw () {
|
||||
}
|
||||
|
||||
virtual const char* what() const throw () {
|
||||
return _reason.c_str();
|
||||
}
|
||||
};
|
||||
|
||||
class OutOfMemoryException : public HadoopException {
|
||||
public:
|
||||
OutOfMemoryException(const string & what)
|
||||
: HadoopException(what) {
|
||||
}
|
||||
};
|
||||
|
||||
class IOException : public HadoopException {
|
||||
public:
|
||||
IOException(const string & what)
|
||||
: HadoopException(what) {
|
||||
}
|
||||
};
|
||||
|
||||
class UnsupportException : public HadoopException {
|
||||
public:
|
||||
UnsupportException(const string & what)
|
||||
: HadoopException(what) {
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Exception when call java methods using JNI
|
||||
*/
|
||||
class JavaException : public HadoopException {
|
||||
public:
|
||||
JavaException(const string & what)
|
||||
: HadoopException(what) {
|
||||
}
|
||||
};
|
||||
|
||||
#define STRINGIFY(x) #x
|
||||
#define TOSTRING(x) STRINGIFY(x)
|
||||
#define AT __FILE__ ":" TOSTRING(__LINE__)
|
||||
#define THROW_EXCEPTION(type, what) throw type((std::string(AT":") + what))
|
||||
#define THROW_EXCEPTION_EX(type, fmt, args...) \
|
||||
throw type(StringUtil::Format("%s:" fmt, AT, ##args))
|
||||
|
||||
class Config {
|
||||
protected:
|
||||
map<string, string> _configs;
|
||||
public:
|
||||
Config() {
|
||||
}
|
||||
~Config() {
|
||||
}
|
||||
|
||||
const char * get(const string & name);
|
||||
|
||||
string get(const string & name, const string & defaultValue);
|
||||
|
||||
bool getBool(const string & name, bool defaultValue);
|
||||
|
||||
int64_t getInt(const string & name, int64_t defaultValue = -1);
|
||||
|
||||
float getFloat(const string & name, float defaultValue = -1);
|
||||
|
||||
void getStrings(const string & name, vector<string> & dest);
|
||||
|
||||
void getInts(const string & name, vector<int64_t> & dest);
|
||||
|
||||
void getFloats(const string & name, vector<float> & dest);
|
||||
|
||||
void set(const string & key, const string & value);
|
||||
|
||||
void setInt(const string & name, int64_t value);
|
||||
|
||||
void setBool(const string & name, bool value);
|
||||
|
||||
/**
|
||||
* Load configs from a config file with the following format:
|
||||
* # comment
|
||||
* key1=value1
|
||||
* key2=value2
|
||||
* ...
|
||||
*/
|
||||
void load(const string & path);
|
||||
|
||||
/**
|
||||
* Load configs form command line args
|
||||
* key1=value1 key2=value2,value2
|
||||
*/
|
||||
void parse(int32_t argc, const char ** argv);
|
||||
};
|
||||
|
||||
class Command {
|
||||
private:
|
||||
int _id;
|
||||
const char * _description;
|
||||
|
||||
public:
|
||||
Command(int id, const char * description)
|
||||
: _id(id), _description(description) {
|
||||
}
|
||||
|
||||
Command(int id)
|
||||
: _id(id), _description(NULL) {
|
||||
}
|
||||
|
||||
int id() const {
|
||||
return _id;
|
||||
}
|
||||
|
||||
const char * description() const {
|
||||
return _description;
|
||||
}
|
||||
|
||||
bool equals(const Command & other) const {
|
||||
if (_id == other._id) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
class Buffer {
|
||||
protected:
|
||||
const char * _data;
|
||||
uint32_t _length;
|
||||
|
||||
public:
|
||||
Buffer()
|
||||
: _data(NULL), _length(0) {
|
||||
}
|
||||
|
||||
Buffer(const char * data, uint32_t length)
|
||||
: _data(data), _length(length) {
|
||||
}
|
||||
|
||||
~Buffer() {
|
||||
}
|
||||
|
||||
void reset(const char * data, uint32_t length) {
|
||||
this->_data = data;
|
||||
this->_length = length;
|
||||
}
|
||||
|
||||
const char * data() const {
|
||||
return _data;
|
||||
}
|
||||
|
||||
uint32_t length() const {
|
||||
return _length;
|
||||
}
|
||||
|
||||
void data(const char * data) {
|
||||
this->_data = data;
|
||||
}
|
||||
|
||||
void length(uint32_t length) {
|
||||
this->_length = length;
|
||||
}
|
||||
|
||||
string toString() const {
|
||||
return string(_data, _length);
|
||||
}
|
||||
};
|
||||
|
||||
class InputSplit {
|
||||
public:
|
||||
virtual uint64_t getLength() = 0;
|
||||
virtual vector<string> & getLocations() = 0;
|
||||
virtual void readFields(const string & data) = 0;
|
||||
virtual void writeFields(string & dest) = 0;
|
||||
virtual string toString() = 0;
|
||||
|
||||
virtual ~InputSplit() {
|
||||
|
||||
}
|
||||
};
|
||||
|
||||
class Configurable : public NativeObject {
|
||||
public:
|
||||
Configurable() {
|
||||
}
|
||||
|
||||
virtual void configure(Config * config) {
|
||||
}
|
||||
};
|
||||
|
||||
class Collector {
|
||||
public:
|
||||
virtual ~Collector() {
|
||||
}
|
||||
|
||||
virtual void collect(const void * key, uint32_t keyLen, const void * value, uint32_t valueLen) {
|
||||
}
|
||||
|
||||
virtual void collect(const void * key, uint32_t keyLen, const void * value, uint32_t valueLen,
|
||||
int32_t partition) {
|
||||
collect(key, keyLen, value, valueLen);
|
||||
}
|
||||
};
|
||||
|
||||
class Progress {
|
||||
public:
|
||||
virtual ~Progress() {
|
||||
}
|
||||
virtual float getProgress() = 0;
|
||||
};
|
||||
|
||||
class Counter {
|
||||
private:
|
||||
// not thread safe
|
||||
// TODO: use atomic
|
||||
volatile uint64_t _count;
|
||||
|
||||
string _group;
|
||||
string _name;
|
||||
public:
|
||||
Counter(const string & group, const string & name)
|
||||
: _count(0), _group(group), _name(name) {
|
||||
}
|
||||
|
||||
const string & group() const {
|
||||
return _group;
|
||||
}
|
||||
const string & name() const {
|
||||
return _name;
|
||||
}
|
||||
|
||||
uint64_t get() const {
|
||||
return _count;
|
||||
}
|
||||
|
||||
void increase() {
|
||||
_count++;
|
||||
}
|
||||
|
||||
void increase(uint64_t cnt) {
|
||||
_count += cnt;
|
||||
}
|
||||
};
|
||||
|
||||
class KVIterator {
|
||||
public:
|
||||
virtual ~KVIterator() {
|
||||
}
|
||||
virtual bool next(Buffer & key, Buffer & value) = 0;
|
||||
};
|
||||
|
||||
|
||||
class ProcessorBase : public Configurable {
|
||||
protected:
|
||||
Collector * _collector;
|
||||
public:
|
||||
ProcessorBase()
|
||||
: _collector(NULL) {
|
||||
}
|
||||
|
||||
void setCollector(Collector * collector) {
|
||||
_collector = collector;
|
||||
}
|
||||
|
||||
Collector * getCollector() {
|
||||
return _collector;
|
||||
}
|
||||
|
||||
void collect(const void * key, uint32_t keyLen, const void * value, uint32_t valueLen) {
|
||||
_collector->collect(key, keyLen, value, valueLen);
|
||||
}
|
||||
|
||||
void collect(const void * key, uint32_t keyLen, const void * value, uint32_t valueLen,
|
||||
int32_t partition) {
|
||||
_collector->collect(key, keyLen, value, valueLen, partition);
|
||||
}
|
||||
|
||||
Counter * getCounter(const string & group, const string & name);
|
||||
|
||||
virtual void close() {
|
||||
}
|
||||
};
|
||||
|
||||
enum KeyGroupIterState {
|
||||
SAME_KEY,
|
||||
NEW_KEY,
|
||||
NEW_KEY_VALUE,
|
||||
NO_MORE,
|
||||
};
|
||||
|
||||
class KeyGroupIterator {
|
||||
public:
|
||||
virtual ~KeyGroupIterator() {
|
||||
}
|
||||
/**
|
||||
* Move to nextKey, or begin this iterator
|
||||
*/
|
||||
virtual bool nextKey() = 0;
|
||||
|
||||
/**
|
||||
* Get key of this input group
|
||||
*/
|
||||
virtual const char * getKey(uint32_t & len) = 0;
|
||||
|
||||
/**
|
||||
* Get next value of this input group
|
||||
* @return NULL if no more
|
||||
*/
|
||||
virtual const char * nextValue(uint32_t & len) = 0;
|
||||
};
|
||||
|
||||
|
||||
|
||||
enum KeyValueType {
|
||||
TextType = 0,
|
||||
BytesType = 1,
|
||||
ByteType = 2,
|
||||
BoolType = 3,
|
||||
IntType = 4,
|
||||
LongType = 5,
|
||||
FloatType = 6,
|
||||
DoubleType = 7,
|
||||
MD5HashType = 8,
|
||||
VIntType = 9,
|
||||
VLongType = 10,
|
||||
UnknownType = -1
|
||||
};
|
||||
|
||||
typedef int (*ComparatorPtr)(const char * src, uint32_t srcLength, const char * dest,
|
||||
uint32_t destLength);
|
||||
|
||||
ComparatorPtr get_comparator(const KeyValueType keyType, const char * comparatorName);
|
||||
|
||||
typedef void (*ANY_FUNC_PTR)();
|
||||
|
||||
} // namespace NativeTask;
|
||||
|
||||
/**
|
||||
* Use these two predefined macro to define a class library:
|
||||
* DEFINE_NATIVE_LIBRARY(Library)
|
||||
* REGISTER_CLASS(Type, Library)
|
||||
* For example, suppose we have a demo application, which has
|
||||
* defined class MyDemoMapper and MyDemoReducer, to register
|
||||
* this module & these two classes, you need to add following
|
||||
* code to you source code.
|
||||
* DEFINE_NATIVE_LIBRARY(MyDemo) {
|
||||
* REGISTER_CLASS(MyDemoMapper, MyDemo);
|
||||
* REGISTER_CLASS(MyDemoReducer, MyDemo);
|
||||
* }
|
||||
* The class name for MyDemoMapper will be MyDemo.MyDemoMapper,
|
||||
* and similar for MyDemoReducer.
|
||||
* Then you can set native.mapper.class to MyDemo.MyDemoMapper
|
||||
* in JobConf.
|
||||
*/
|
||||
|
||||
#define DEFINE_NATIVE_LIBRARY(Library) \
|
||||
static std::map<std::string, NativeTask::ObjectCreatorFunc> Library##ClassMap__; \
|
||||
extern "C" void * Library##GetFunctionGetter(const std::string & name) { \
|
||||
std::map<std::string, NativeTask::ObjectCreatorFunc>::iterator itr = Library##ClassMap__.find(name); \
|
||||
if (itr != Library##ClassMap__.end()) { \
|
||||
return (void *)(itr->second); \
|
||||
} \
|
||||
return NULL; \
|
||||
} \
|
||||
extern "C" NativeTask::ObjectCreatorFunc Library##GetObjectCreator(const std::string & name) { \
|
||||
std::map<std::string, NativeTask::ObjectCreatorFunc>::iterator itr = Library##ClassMap__.find(name); \
|
||||
if (itr != Library##ClassMap__.end()) { \
|
||||
return itr->second; \
|
||||
} \
|
||||
return NULL; \
|
||||
} \
|
||||
extern "C" void Library##Init()
|
||||
|
||||
#define REGISTER_CLASS(Type, Library) Library##ClassMap__[#Library"."#Type] = NativeTask::ObjectCreator<Type>
|
||||
|
||||
#define REGISTER_FUNCTION(Type, Library) Library##ClassMap__[#Library"."#Type] = (ObjectCreatorFunc)Type
|
||||
|
||||
#endif /* NATIVETASK_H_ */
|
|
@ -0,0 +1,179 @@
|
|||
/**
|
||||
* 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 "lib/commons.h"
|
||||
#include "NativeTask.h"
|
||||
#include "BlockCodec.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
BlockCompressStream::BlockCompressStream(OutputStream * stream, uint32_t bufferSizeHint)
|
||||
: CompressStream(stream), _tempBuffer(NULL), _tempBufferSize(0), _compressedBytesWritten(0) {
|
||||
_hint = bufferSizeHint;
|
||||
_blockMax = bufferSizeHint / 2 * 3;
|
||||
}
|
||||
|
||||
void BlockCompressStream::init() {
|
||||
_tempBufferSize = maxCompressedLength(_blockMax) + 8;
|
||||
_tempBuffer = new char[_tempBufferSize];
|
||||
}
|
||||
|
||||
BlockCompressStream::~BlockCompressStream() {
|
||||
delete[] _tempBuffer;
|
||||
_tempBuffer = NULL;
|
||||
_tempBufferSize = 0;
|
||||
}
|
||||
|
||||
void BlockCompressStream::write(const void * buff, uint32_t length) {
|
||||
while (length > 0) {
|
||||
uint32_t take = length < _blockMax ? length : _hint;
|
||||
compressOneBlock(buff, take);
|
||||
buff = ((const char *)buff) + take;
|
||||
length -= take;
|
||||
}
|
||||
}
|
||||
|
||||
void BlockCompressStream::flush() {
|
||||
_stream->flush();
|
||||
}
|
||||
|
||||
void BlockCompressStream::close() {
|
||||
flush();
|
||||
}
|
||||
|
||||
void BlockCompressStream::writeDirect(const void * buff, uint32_t length) {
|
||||
_stream->write(buff, length);
|
||||
_compressedBytesWritten += length;
|
||||
}
|
||||
|
||||
uint64_t BlockCompressStream::compressedBytesWritten() {
|
||||
return _compressedBytesWritten;
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////
|
||||
|
||||
BlockDecompressStream::BlockDecompressStream(InputStream * stream, uint32_t bufferSizeHint)
|
||||
: DecompressStream(stream), _tempBuffer(NULL), _tempBufferSize(0) {
|
||||
_hint = bufferSizeHint;
|
||||
_blockMax = bufferSizeHint / 2 * 3;
|
||||
_tempDecompressBuffer = NULL;
|
||||
_tempDecompressBufferSize = 0;
|
||||
_tempDecompressBufferUsed = 0;
|
||||
_tempDecompressBufferCapacity = 0;
|
||||
_compressedBytesRead = 0;
|
||||
}
|
||||
|
||||
void BlockDecompressStream::init() {
|
||||
_tempBufferSize = maxCompressedLength(_blockMax) + 8;
|
||||
_tempBuffer = (char*)malloc(_tempBufferSize);
|
||||
}
|
||||
|
||||
BlockDecompressStream::~BlockDecompressStream() {
|
||||
close();
|
||||
if (NULL != _tempBuffer) {
|
||||
free(_tempBuffer);
|
||||
_tempBuffer = NULL;
|
||||
}
|
||||
_tempBufferSize = 0;
|
||||
}
|
||||
|
||||
int32_t BlockDecompressStream::read(void * buff, uint32_t length) {
|
||||
if (_tempDecompressBufferSize == 0) {
|
||||
uint32_t sizes[2];
|
||||
int32_t rd = _stream->readFully(&sizes, sizeof(uint32_t) * 2);
|
||||
if (rd <= 0) {
|
||||
// EOF
|
||||
return -1;
|
||||
}
|
||||
if (rd != sizeof(uint32_t) * 2) {
|
||||
THROW_EXCEPTION(IOException, "readFully get incomplete data");
|
||||
}
|
||||
_compressedBytesRead += rd;
|
||||
sizes[0] = bswap(sizes[0]);
|
||||
sizes[1] = bswap(sizes[1]);
|
||||
if (sizes[0] <= length) {
|
||||
uint32_t len = decompressOneBlock(sizes[1], buff, sizes[0]);
|
||||
if (len != sizes[0]) {
|
||||
THROW_EXCEPTION(IOException, "Block decompress data error, length not match");
|
||||
}
|
||||
return len;
|
||||
} else {
|
||||
if (sizes[0] > _tempDecompressBufferCapacity) {
|
||||
char * newBuffer = (char *)realloc(_tempDecompressBuffer, sizes[0]);
|
||||
if (newBuffer == NULL) {
|
||||
THROW_EXCEPTION(OutOfMemoryException, "realloc failed");
|
||||
}
|
||||
_tempDecompressBuffer = newBuffer;
|
||||
_tempDecompressBufferCapacity = sizes[0];
|
||||
}
|
||||
uint32_t len = decompressOneBlock(sizes[1], _tempDecompressBuffer, sizes[0]);
|
||||
if (len != sizes[0]) {
|
||||
THROW_EXCEPTION(IOException, "Block decompress data error, length not match");
|
||||
}
|
||||
_tempDecompressBufferSize = sizes[0];
|
||||
_tempDecompressBufferUsed = 0;
|
||||
}
|
||||
}
|
||||
if (_tempDecompressBufferSize > 0) {
|
||||
uint32_t left = _tempDecompressBufferSize - _tempDecompressBufferUsed;
|
||||
if (length < left) {
|
||||
memcpy(buff, _tempDecompressBuffer + _tempDecompressBufferUsed, length);
|
||||
_tempDecompressBufferUsed += length;
|
||||
return length;
|
||||
} else {
|
||||
memcpy(buff, _tempDecompressBuffer + _tempDecompressBufferUsed, left);
|
||||
_tempDecompressBufferSize = 0;
|
||||
_tempDecompressBufferUsed = 0;
|
||||
return left;
|
||||
}
|
||||
}
|
||||
// should not get here
|
||||
THROW_EXCEPTION(IOException, "Decompress logic error");
|
||||
return -1;
|
||||
}
|
||||
|
||||
void BlockDecompressStream::close() {
|
||||
if (_tempDecompressBufferSize > 0) {
|
||||
LOG("[BlockDecompressStream] Some data left in the _tempDecompressBuffer when close()");
|
||||
}
|
||||
if (NULL != _tempDecompressBuffer) {
|
||||
free(_tempDecompressBuffer);
|
||||
_tempDecompressBuffer = NULL;
|
||||
_tempDecompressBufferCapacity = 0;
|
||||
}
|
||||
_tempDecompressBufferSize = 0;
|
||||
_tempDecompressBufferUsed = 0;
|
||||
}
|
||||
|
||||
int32_t BlockDecompressStream::readDirect(void * buff, uint32_t length) {
|
||||
if (_tempDecompressBufferSize > 0) {
|
||||
THROW_EXCEPTION(IOException, "temp decompress data exists when call readDirect()");
|
||||
}
|
||||
int32_t ret = _stream->readFully(buff, length);
|
||||
if (ret > 0) {
|
||||
_compressedBytesRead += ret;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
uint64_t BlockDecompressStream::compressedBytesRead() {
|
||||
return _compressedBytesRead;
|
||||
}
|
||||
|
||||
} // namespace NativeTask
|
||||
|
|
@ -0,0 +1,98 @@
|
|||
/**
|
||||
* 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 BLOCKCODEC_H_
|
||||
#define BLOCKCODEC_H_
|
||||
|
||||
#include "lib/Compressions.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
class BlockCompressStream : public CompressStream {
|
||||
protected:
|
||||
uint32_t _hint;
|
||||
uint32_t _blockMax;
|
||||
char * _tempBuffer;
|
||||
uint32_t _tempBufferSize;
|
||||
uint64_t _compressedBytesWritten;
|
||||
public:
|
||||
BlockCompressStream(OutputStream * stream, uint32_t bufferSizeHint);
|
||||
|
||||
virtual ~BlockCompressStream();
|
||||
|
||||
virtual void write(const void * buff, uint32_t length);
|
||||
|
||||
virtual void flush();
|
||||
|
||||
virtual void close();
|
||||
|
||||
virtual void writeDirect(const void * buff, uint32_t length);
|
||||
|
||||
virtual uint64_t compressedBytesWritten();
|
||||
|
||||
void init();
|
||||
|
||||
protected:
|
||||
virtual uint64_t maxCompressedLength(uint64_t origLength) {
|
||||
return origLength;
|
||||
}
|
||||
|
||||
virtual void compressOneBlock(const void * buff, uint32_t length) {
|
||||
}
|
||||
};
|
||||
|
||||
class BlockDecompressStream : public DecompressStream {
|
||||
protected:
|
||||
uint32_t _hint;
|
||||
uint32_t _blockMax;
|
||||
char * _tempBuffer;
|
||||
uint32_t _tempBufferSize;
|
||||
char * _tempDecompressBuffer;
|
||||
uint32_t _tempDecompressBufferSize;
|
||||
uint32_t _tempDecompressBufferUsed;
|
||||
uint32_t _tempDecompressBufferCapacity;
|
||||
uint64_t _compressedBytesRead;
|
||||
public:
|
||||
BlockDecompressStream(InputStream * stream, uint32_t bufferSizeHint);
|
||||
|
||||
virtual ~BlockDecompressStream();
|
||||
|
||||
virtual int32_t read(void * buff, uint32_t length);
|
||||
|
||||
virtual void close();
|
||||
|
||||
virtual int32_t readDirect(void * buff, uint32_t length);
|
||||
|
||||
virtual uint64_t compressedBytesRead();
|
||||
|
||||
void init();
|
||||
|
||||
protected:
|
||||
virtual uint64_t maxCompressedLength(uint64_t origLength) {
|
||||
return origLength;
|
||||
}
|
||||
|
||||
virtual uint32_t decompressOneBlock(uint32_t compressedSize, void * buff, uint32_t length) {
|
||||
//TODO: add implementation
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace NativeTask
|
||||
|
||||
#endif /* BLOCKCODEC_H_ */
|
|
@ -0,0 +1,192 @@
|
|||
/**
|
||||
* 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 <zconf.h>
|
||||
#include <zlib.h>
|
||||
#include "lib/commons.h"
|
||||
#include "GzipCodec.h"
|
||||
#include <iostream>
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
GzipCompressStream::GzipCompressStream(OutputStream * stream, uint32_t bufferSizeHint)
|
||||
: CompressStream(stream), _compressedBytesWritten(0), _zstream(NULL), _finished(false) {
|
||||
_buffer = new char[bufferSizeHint];
|
||||
_capacity = bufferSizeHint;
|
||||
_zstream = malloc(sizeof(z_stream));
|
||||
z_stream * zstream = (z_stream*)_zstream;
|
||||
memset(zstream, 0, sizeof(z_stream));
|
||||
if (Z_OK != deflateInit2(zstream, Z_DEFAULT_COMPRESSION, Z_DEFLATED, 31, 8,
|
||||
Z_DEFAULT_STRATEGY)) {
|
||||
free(_zstream);
|
||||
_zstream = NULL;
|
||||
THROW_EXCEPTION(IOException, "deflateInit2 failed");
|
||||
}
|
||||
zstream->next_out = (Bytef *)_buffer;
|
||||
zstream->avail_out = _capacity;
|
||||
}
|
||||
|
||||
GzipCompressStream::~GzipCompressStream() {
|
||||
if (_zstream != NULL) {
|
||||
deflateEnd((z_stream*)_zstream);
|
||||
free(_zstream);
|
||||
_zstream = NULL;
|
||||
}
|
||||
delete[] _buffer;
|
||||
_buffer = NULL;
|
||||
}
|
||||
|
||||
void GzipCompressStream::write(const void * buff, uint32_t length) {
|
||||
z_stream * zstream = (z_stream*)_zstream;
|
||||
zstream->next_in = (Bytef*)buff;
|
||||
zstream->avail_in = length;
|
||||
while (true) {
|
||||
int ret = deflate(zstream, Z_NO_FLUSH);
|
||||
if (ret == Z_OK) {
|
||||
if (zstream->avail_out == 0) {
|
||||
_stream->write(_buffer, _capacity);
|
||||
_compressedBytesWritten += _capacity;
|
||||
zstream->next_out = (Bytef *)_buffer;
|
||||
zstream->avail_out = _capacity;
|
||||
}
|
||||
if (zstream->avail_in == 0) {
|
||||
break;
|
||||
}
|
||||
} else {
|
||||
THROW_EXCEPTION(IOException, "deflate return error");
|
||||
}
|
||||
}
|
||||
_finished = false;
|
||||
}
|
||||
|
||||
void GzipCompressStream::flush() {
|
||||
z_stream * zstream = (z_stream*)_zstream;
|
||||
while (true) {
|
||||
int ret = deflate(zstream, Z_FINISH);
|
||||
if (ret == Z_OK) {
|
||||
if (zstream->avail_out == 0) {
|
||||
_stream->write(_buffer, _capacity);
|
||||
_compressedBytesWritten += _capacity;
|
||||
zstream->next_out = (Bytef *)_buffer;
|
||||
zstream->avail_out = _capacity;
|
||||
} else {
|
||||
THROW_EXCEPTION(IOException, "flush state error");
|
||||
}
|
||||
} else if (ret == Z_STREAM_END) {
|
||||
size_t wt = zstream->next_out - (Bytef*)_buffer;
|
||||
_stream->write(_buffer, wt);
|
||||
_compressedBytesWritten += wt;
|
||||
zstream->next_out = (Bytef *)_buffer;
|
||||
zstream->avail_out = _capacity;
|
||||
break;
|
||||
}
|
||||
}
|
||||
_finished = true;
|
||||
_stream->flush();
|
||||
}
|
||||
|
||||
void GzipCompressStream::resetState() {
|
||||
z_stream * zstream = (z_stream*)_zstream;
|
||||
deflateReset(zstream);
|
||||
}
|
||||
|
||||
void GzipCompressStream::close() {
|
||||
if (!_finished) {
|
||||
flush();
|
||||
}
|
||||
}
|
||||
|
||||
void GzipCompressStream::writeDirect(const void * buff, uint32_t length) {
|
||||
if (!_finished) {
|
||||
flush();
|
||||
}
|
||||
_stream->write(buff, length);
|
||||
_compressedBytesWritten += length;
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////
|
||||
|
||||
GzipDecompressStream::GzipDecompressStream(InputStream * stream, uint32_t bufferSizeHint)
|
||||
: DecompressStream(stream), _compressedBytesRead(0), _zstream(NULL) {
|
||||
_buffer = new char[bufferSizeHint];
|
||||
_capacity = bufferSizeHint;
|
||||
_zstream = malloc(sizeof(z_stream));
|
||||
z_stream * zstream = (z_stream*)_zstream;
|
||||
memset(zstream, 0, sizeof(z_stream));
|
||||
if (Z_OK != inflateInit2(zstream, 31)) {
|
||||
free(_zstream);
|
||||
_zstream = NULL;
|
||||
THROW_EXCEPTION(IOException, "inflateInit2 failed");
|
||||
}
|
||||
zstream->next_in = NULL;
|
||||
zstream->avail_in = 0;
|
||||
_eof = false;
|
||||
}
|
||||
|
||||
GzipDecompressStream::~GzipDecompressStream() {
|
||||
if (_zstream != NULL) {
|
||||
inflateEnd((z_stream*)_zstream);
|
||||
free(_zstream);
|
||||
_zstream = NULL;
|
||||
}
|
||||
delete[] _buffer;
|
||||
_buffer = NULL;
|
||||
}
|
||||
|
||||
int32_t GzipDecompressStream::read(void * buff, uint32_t length) {
|
||||
z_stream * zstream = (z_stream*)_zstream;
|
||||
zstream->next_out = (Bytef*)buff;
|
||||
zstream->avail_out = length;
|
||||
while (true) {
|
||||
if (zstream->avail_in == 0) {
|
||||
int32_t rd = _stream->read(_buffer, _capacity);
|
||||
if (rd <= 0) {
|
||||
_eof = true;
|
||||
size_t wt = zstream->next_out - (Bytef*)buff;
|
||||
return wt > 0 ? wt : -1;
|
||||
} else {
|
||||
_compressedBytesRead += rd;
|
||||
zstream->next_in = (Bytef*)_buffer;
|
||||
zstream->avail_in = rd;
|
||||
}
|
||||
}
|
||||
int ret = inflate(zstream, Z_NO_FLUSH);
|
||||
if (ret == Z_OK || ret == Z_STREAM_END) {
|
||||
if (zstream->avail_out == 0) {
|
||||
return length;
|
||||
}
|
||||
} else {
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
return -1;
|
||||
}
|
||||
|
||||
void GzipDecompressStream::close() {
|
||||
}
|
||||
|
||||
int32_t GzipDecompressStream::readDirect(void * buff, uint32_t length) {
|
||||
int32_t ret = _stream->readFully(buff, length);
|
||||
if (ret > 0) {
|
||||
_compressedBytesRead += ret;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace NativeTask
|
||||
|
|
@ -0,0 +1,82 @@
|
|||
/**
|
||||
* 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 GZIPCODEC_H_
|
||||
#define GZIPCODEC_H_
|
||||
|
||||
#include "lib/Compressions.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
class GzipCompressStream : public CompressStream {
|
||||
protected:
|
||||
uint64_t _compressedBytesWritten;
|
||||
char * _buffer;
|
||||
uint32_t _capacity;
|
||||
void * _zstream;
|
||||
bool _finished;
|
||||
public:
|
||||
GzipCompressStream(OutputStream * stream, uint32_t bufferSizeHint);
|
||||
|
||||
virtual ~GzipCompressStream();
|
||||
|
||||
virtual void write(const void * buff, uint32_t length);
|
||||
|
||||
virtual void flush();
|
||||
|
||||
virtual void close();
|
||||
|
||||
virtual void finish() {
|
||||
flush();
|
||||
}
|
||||
|
||||
virtual void resetState();
|
||||
|
||||
virtual void writeDirect(const void * buff, uint32_t length);
|
||||
|
||||
virtual uint64_t compressedBytesWritten() {
|
||||
return _compressedBytesWritten;
|
||||
}
|
||||
};
|
||||
|
||||
class GzipDecompressStream : public DecompressStream {
|
||||
protected:
|
||||
uint64_t _compressedBytesRead;
|
||||
char * _buffer;
|
||||
uint32_t _capacity;
|
||||
void * _zstream;
|
||||
bool _eof;
|
||||
public:
|
||||
GzipDecompressStream(InputStream * stream, uint32_t bufferSizeHint);
|
||||
|
||||
virtual ~GzipDecompressStream();
|
||||
|
||||
virtual int32_t read(void * buff, uint32_t length);
|
||||
|
||||
virtual void close();
|
||||
|
||||
virtual int32_t readDirect(void * buff, uint32_t length);
|
||||
|
||||
virtual uint64_t compressedBytesRead() {
|
||||
return _compressedBytesRead;
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace NativeTask
|
||||
|
||||
#endif /* GZIPCODEC_H_ */
|
|
@ -0,0 +1,88 @@
|
|||
/**
|
||||
* 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 "lib/commons.h"
|
||||
#include "lz4.h"
|
||||
#include "NativeTask.h"
|
||||
#include "Lz4Codec.h"
|
||||
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
static int32_t LZ4_MaxCompressedSize(int32_t orig) {
|
||||
return LZ4_compressBound(orig);
|
||||
}
|
||||
|
||||
Lz4CompressStream::Lz4CompressStream(OutputStream * stream, uint32_t bufferSizeHint)
|
||||
: BlockCompressStream(stream, bufferSizeHint) {
|
||||
init();
|
||||
}
|
||||
|
||||
void Lz4CompressStream::compressOneBlock(const void * buff, uint32_t length) {
|
||||
size_t compressedLength = _tempBufferSize - 8;
|
||||
int ret = LZ4_compress((char*)buff, _tempBuffer + 8, length);
|
||||
if (ret > 0) {
|
||||
compressedLength = ret;
|
||||
((uint32_t*)_tempBuffer)[0] = bswap(length);
|
||||
((uint32_t*)_tempBuffer)[1] = bswap((uint32_t)compressedLength);
|
||||
_stream->write(_tempBuffer, compressedLength + 8);
|
||||
_compressedBytesWritten += (compressedLength + 8);
|
||||
} else {
|
||||
THROW_EXCEPTION(IOException, "compress LZ4 failed");
|
||||
}
|
||||
}
|
||||
|
||||
uint64_t Lz4CompressStream::maxCompressedLength(uint64_t origLength) {
|
||||
return LZ4_MaxCompressedSize(origLength);
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////
|
||||
|
||||
Lz4DecompressStream::Lz4DecompressStream(InputStream * stream, uint32_t bufferSizeHint)
|
||||
: BlockDecompressStream(stream, bufferSizeHint) {
|
||||
init();
|
||||
}
|
||||
|
||||
uint32_t Lz4DecompressStream::decompressOneBlock(uint32_t compressedSize, void * buff,
|
||||
uint32_t length) {
|
||||
if (compressedSize > _tempBufferSize) {
|
||||
char * newBuffer = (char *)realloc(_tempBuffer, compressedSize);
|
||||
if (newBuffer == NULL) {
|
||||
THROW_EXCEPTION(OutOfMemoryException, "realloc failed");
|
||||
}
|
||||
_tempBuffer = newBuffer;
|
||||
_tempBufferSize = compressedSize;
|
||||
}
|
||||
uint32_t rd = _stream->readFully(_tempBuffer, compressedSize);
|
||||
if (rd != compressedSize) {
|
||||
THROW_EXCEPTION(IOException, "readFully reach EOF");
|
||||
}
|
||||
_compressedBytesRead += rd;
|
||||
uint32_t ret = LZ4_uncompress(_tempBuffer, (char*)buff, length);
|
||||
if (ret == compressedSize) {
|
||||
return length;
|
||||
} else {
|
||||
THROW_EXCEPTION(IOException, "decompress LZ4 failed");
|
||||
}
|
||||
}
|
||||
|
||||
uint64_t Lz4DecompressStream::maxCompressedLength(uint64_t origLength) {
|
||||
return LZ4_MaxCompressedSize(origLength);
|
||||
}
|
||||
|
||||
} // namespace NativeTask
|
|
@ -0,0 +1,45 @@
|
|||
/**
|
||||
* 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 LZ4CODEC_H_
|
||||
#define LZ4CODEC_H_
|
||||
|
||||
#include "lib/Compressions.h"
|
||||
#include "BlockCodec.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
class Lz4CompressStream : public BlockCompressStream {
|
||||
public:
|
||||
Lz4CompressStream(OutputStream * stream, uint32_t bufferSizeHint);
|
||||
protected:
|
||||
virtual uint64_t maxCompressedLength(uint64_t origLength);
|
||||
virtual void compressOneBlock(const void * buff, uint32_t length);
|
||||
};
|
||||
|
||||
class Lz4DecompressStream : public BlockDecompressStream {
|
||||
public:
|
||||
Lz4DecompressStream(InputStream * stream, uint32_t bufferSizeHint);
|
||||
protected:
|
||||
virtual uint64_t maxCompressedLength(uint64_t origLength);
|
||||
virtual uint32_t decompressOneBlock(uint32_t compressedSize, void * buff, uint32_t length);
|
||||
};
|
||||
|
||||
} // namespace NativeTask
|
||||
|
||||
#endif /* LZ4CODEC_H_ */
|
|
@ -0,0 +1,98 @@
|
|||
/**
|
||||
* 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 "config.h"
|
||||
|
||||
#if defined HADOOP_SNAPPY_LIBRARY
|
||||
#include "lib/commons.h"
|
||||
#include "NativeTask.h"
|
||||
#include "SnappyCodec.h"
|
||||
|
||||
#include <snappy-c.h>
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
SnappyCompressStream::SnappyCompressStream(OutputStream * stream, uint32_t bufferSizeHint)
|
||||
: BlockCompressStream(stream, bufferSizeHint) {
|
||||
init();
|
||||
}
|
||||
|
||||
void SnappyCompressStream::compressOneBlock(const void * buff, uint32_t length) {
|
||||
size_t compressedLength = _tempBufferSize - 8;
|
||||
snappy_status ret = snappy_compress((const char*)buff, length, _tempBuffer + 8,
|
||||
&compressedLength);
|
||||
if (ret == SNAPPY_OK) {
|
||||
((uint32_t*)_tempBuffer)[0] = bswap(length);
|
||||
((uint32_t*)_tempBuffer)[1] = bswap((uint32_t)compressedLength);
|
||||
_stream->write(_tempBuffer, compressedLength + 8);
|
||||
_compressedBytesWritten += (compressedLength + 8);
|
||||
} else if (ret == SNAPPY_INVALID_INPUT) {
|
||||
THROW_EXCEPTION(IOException, "compress SNAPPY_INVALID_INPUT");
|
||||
} else if (ret == SNAPPY_BUFFER_TOO_SMALL) {
|
||||
THROW_EXCEPTION(IOException, "compress SNAPPY_BUFFER_TOO_SMALL");
|
||||
} else {
|
||||
THROW_EXCEPTION(IOException, "compress snappy failed");
|
||||
}
|
||||
}
|
||||
|
||||
uint64_t SnappyCompressStream::maxCompressedLength(uint64_t origLength) {
|
||||
return snappy_max_compressed_length(origLength);
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////
|
||||
|
||||
SnappyDecompressStream::SnappyDecompressStream(InputStream * stream, uint32_t bufferSizeHint)
|
||||
: BlockDecompressStream(stream, bufferSizeHint) {
|
||||
init();
|
||||
}
|
||||
|
||||
uint32_t SnappyDecompressStream::decompressOneBlock(uint32_t compressedSize, void * buff,
|
||||
uint32_t length) {
|
||||
if (compressedSize > _tempBufferSize) {
|
||||
char * newBuffer = (char *)realloc(_tempBuffer, compressedSize);
|
||||
if (newBuffer == NULL) {
|
||||
THROW_EXCEPTION(OutOfMemoryException, "realloc failed");
|
||||
}
|
||||
_tempBuffer = newBuffer;
|
||||
_tempBufferSize = compressedSize;
|
||||
}
|
||||
uint32_t rd = _stream->readFully(_tempBuffer, compressedSize);
|
||||
if (rd != compressedSize) {
|
||||
THROW_EXCEPTION(IOException, "readFully reach EOF");
|
||||
}
|
||||
_compressedBytesRead += rd;
|
||||
size_t uncompressedLength = length;
|
||||
snappy_status ret = snappy_uncompress(_tempBuffer, compressedSize, (char *)buff,
|
||||
&uncompressedLength);
|
||||
if (ret == SNAPPY_OK) {
|
||||
return uncompressedLength;
|
||||
} else if (ret == SNAPPY_INVALID_INPUT) {
|
||||
THROW_EXCEPTION(IOException, "decompress SNAPPY_INVALID_INPUT");
|
||||
} else if (ret == SNAPPY_BUFFER_TOO_SMALL) {
|
||||
THROW_EXCEPTION(IOException, "decompress SNAPPY_BUFFER_TOO_SMALL");
|
||||
} else {
|
||||
THROW_EXCEPTION(IOException, "decompress snappy failed");
|
||||
}
|
||||
}
|
||||
|
||||
uint64_t SnappyDecompressStream::maxCompressedLength(uint64_t origLength) {
|
||||
return snappy_max_compressed_length(origLength);
|
||||
}
|
||||
} // namespace NativeTask
|
||||
|
||||
#endif // define HADOOP_SNAPPY_LIBRARY
|
|
@ -0,0 +1,46 @@
|
|||
/**
|
||||
* 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 SNAPPYCODEC_H_
|
||||
#define SNAPPYCODEC_H_
|
||||
|
||||
#include "lib/Compressions.h"
|
||||
#include "BlockCodec.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
class SnappyCompressStream : public BlockCompressStream {
|
||||
public:
|
||||
SnappyCompressStream(OutputStream * stream, uint32_t bufferSizeHint);
|
||||
protected:
|
||||
virtual uint64_t maxCompressedLength(uint64_t origLength);
|
||||
virtual void compressOneBlock(const void * buff, uint32_t length);
|
||||
};
|
||||
|
||||
class SnappyDecompressStream : public BlockDecompressStream {
|
||||
public:
|
||||
SnappyDecompressStream(InputStream * stream, uint32_t bufferSizeHint);
|
||||
|
||||
protected:
|
||||
virtual uint64_t maxCompressedLength(uint64_t origLength);
|
||||
virtual uint32_t decompressOneBlock(uint32_t compressedSize, void * buff, uint32_t length);
|
||||
};
|
||||
|
||||
} // namespace NativeTask
|
||||
|
||||
#endif /* SNAPPYCODEC_H_ */
|
|
@ -0,0 +1,34 @@
|
|||
/*
|
||||
* 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 "lib/commons.h"
|
||||
#include "util/StringUtil.h"
|
||||
#include "MCollectorOutputHandler.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 NativeTask
|
|
@ -0,0 +1,102 @@
|
|||
/*
|
||||
* 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 ABSTRACT_MAP_HANDLER_H
|
||||
#define ABSTRACT_MAP_HANDLER_H
|
||||
|
||||
#include "NativeTask.h"
|
||||
#include "BatchHandler.h"
|
||||
#include "lib/SpillOutputService.h"
|
||||
#include "lib/Combiner.h"
|
||||
#include "CombineHandler.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
class AbstractMapHandler : public BatchHandler, public SpillOutputService {
|
||||
public:
|
||||
static const Command GET_OUTPUT_PATH;
|
||||
static const Command GET_OUTPUT_INDEX_PATH;
|
||||
static const Command GET_SPILL_PATH;
|
||||
static const Command GET_COMBINE_HANDLER;
|
||||
|
||||
public:
|
||||
AbstractMapHandler() {}
|
||||
|
||||
virtual ~AbstractMapHandler() {}
|
||||
|
||||
virtual void configure(Config * config) {
|
||||
_config = config;
|
||||
}
|
||||
|
||||
virtual string * getOutputPath() {
|
||||
ResultBuffer * outputPathResult = call(GET_OUTPUT_PATH, NULL);
|
||||
if (NULL == outputPathResult) {
|
||||
return NULL;
|
||||
}
|
||||
string * outputPath = outputPathResult->readString();
|
||||
|
||||
delete outputPathResult;
|
||||
return outputPath;
|
||||
}
|
||||
|
||||
virtual string * getOutputIndexPath() {
|
||||
|
||||
ResultBuffer * outputIndexPath = call(GET_OUTPUT_INDEX_PATH, NULL);
|
||||
if (NULL == outputIndexPath) {
|
||||
return NULL;
|
||||
}
|
||||
string * indexpath = outputIndexPath->readString();
|
||||
delete outputIndexPath;
|
||||
return indexpath;
|
||||
}
|
||||
|
||||
|
||||
virtual string * getSpillPath() {
|
||||
ResultBuffer * spillPathBuffer = call(GET_SPILL_PATH, NULL);
|
||||
if (NULL == spillPathBuffer) {
|
||||
return NULL;
|
||||
}
|
||||
string * spillpath = spillPathBuffer->readString();
|
||||
delete spillPathBuffer;
|
||||
return spillpath;
|
||||
}
|
||||
|
||||
virtual CombineHandler * getJavaCombineHandler() {
|
||||
|
||||
LOG("[MapOutputCollector::configure] java combiner is configured");
|
||||
|
||||
ResultBuffer * getCombineHandlerResult = call(GET_COMBINE_HANDLER, NULL);
|
||||
if (NULL != getCombineHandlerResult) {
|
||||
|
||||
getCombineHandlerResult->setReadPoint(0);
|
||||
|
||||
CombineHandler * javaCombiner = (CombineHandler *)((BatchHandler * )(getCombineHandlerResult->readPointer()));
|
||||
delete getCombineHandlerResult;
|
||||
return javaCombiner;
|
||||
}
|
||||
|
||||
|
||||
|
||||
return NULL;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
} // namespace NativeTask
|
||||
|
||||
#endif /* MMAPPERHANDLER_H_ */
|
|
@ -0,0 +1,355 @@
|
|||
/*
|
||||
* 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 QUICK_BUILD
|
||||
#include "org_apache_hadoop_mapred_nativetask_NativeBatchProcessor.h"
|
||||
#endif
|
||||
#include "lib/commons.h"
|
||||
#include "jni_md.h"
|
||||
#include "lib/jniutils.h"
|
||||
#include "BatchHandler.h"
|
||||
#include "lib/NativeObjectFactory.h"
|
||||
|
||||
///////////////////////////////////////////////////////////////
|
||||
// NativeBatchProcessor jni util methods
|
||||
///////////////////////////////////////////////////////////////
|
||||
|
||||
static jfieldID InputBufferFieldID = NULL;
|
||||
static jfieldID OutputBufferFieldID = NULL;
|
||||
static jmethodID FlushOutputMethodID = NULL;
|
||||
static jmethodID FinishOutputMethodID = NULL;
|
||||
static jmethodID SendCommandToJavaMethodID = NULL;
|
||||
|
||||
///////////////////////////////////////////////////////////////
|
||||
// BatchHandler methods
|
||||
///////////////////////////////////////////////////////////////
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
ReadWriteBuffer * JNU_ByteArraytoReadWriteBuffer(JNIEnv * jenv, jbyteArray src) {
|
||||
if (NULL == src) {
|
||||
return NULL;
|
||||
}
|
||||
jsize len = jenv->GetArrayLength(src);
|
||||
|
||||
ReadWriteBuffer * ret = new ReadWriteBuffer(len);
|
||||
jenv->GetByteArrayRegion(src, 0, len, (jbyte*)ret->getBuff());
|
||||
ret->setWritePoint(len);
|
||||
return ret;
|
||||
}
|
||||
|
||||
jbyteArray JNU_ReadWriteBufferToByteArray(JNIEnv * jenv, ReadWriteBuffer * result) {
|
||||
if (NULL == result || result->getWritePoint() == 0) {
|
||||
return NULL;
|
||||
}
|
||||
|
||||
jbyteArray ret = jenv->NewByteArray(result->getWritePoint());
|
||||
jenv->SetByteArrayRegion(ret, 0, result->getWritePoint(), (jbyte*)result->getBuff());
|
||||
return ret;
|
||||
}
|
||||
|
||||
BatchHandler::BatchHandler()
|
||||
: _processor(NULL), _config(NULL) {
|
||||
}
|
||||
|
||||
BatchHandler::~BatchHandler() {
|
||||
releaseProcessor();
|
||||
if (NULL != _config) {
|
||||
delete _config;
|
||||
_config = NULL;
|
||||
}
|
||||
}
|
||||
|
||||
void BatchHandler::releaseProcessor() {
|
||||
if (_processor != NULL) {
|
||||
JNIEnv * env = JNU_GetJNIEnv();
|
||||
env->DeleteGlobalRef((jobject)_processor);
|
||||
_processor = NULL;
|
||||
}
|
||||
}
|
||||
|
||||
void BatchHandler::onInputData(uint32_t length) {
|
||||
_in.rewind(0, length);
|
||||
handleInput(_in);
|
||||
}
|
||||
|
||||
void BatchHandler::flushOutput() {
|
||||
|
||||
if (NULL == _out.base()) {
|
||||
return;
|
||||
}
|
||||
|
||||
uint32_t length = _out.position();
|
||||
_out.position(0);
|
||||
|
||||
if (length == 0) {
|
||||
return;
|
||||
}
|
||||
|
||||
JNIEnv * env = JNU_GetJNIEnv();
|
||||
env->CallVoidMethod((jobject)_processor, FlushOutputMethodID, (jint)length);
|
||||
if (env->ExceptionCheck()) {
|
||||
THROW_EXCEPTION(JavaException, "FlushOutput throw exception");
|
||||
}
|
||||
}
|
||||
|
||||
void BatchHandler::finishOutput() {
|
||||
if (NULL == _out.base()) {
|
||||
return;
|
||||
}
|
||||
JNIEnv * env = JNU_GetJNIEnv();
|
||||
env->CallVoidMethod((jobject)_processor, FinishOutputMethodID);
|
||||
if (env->ExceptionCheck()) {
|
||||
THROW_EXCEPTION(JavaException, "FinishOutput throw exception");
|
||||
}
|
||||
}
|
||||
|
||||
void BatchHandler::onSetup(Config * config, char * inputBuffer, uint32_t inputBufferCapacity,
|
||||
char * outputBuffer, uint32_t outputBufferCapacity) {
|
||||
this->_config = config;
|
||||
_in.reset(inputBuffer, inputBufferCapacity);
|
||||
if (NULL != outputBuffer) {
|
||||
if (outputBufferCapacity <= 1024) {
|
||||
THROW_EXCEPTION(IOException, "Output buffer size too small for BatchHandler");
|
||||
}
|
||||
_out.reset(outputBuffer, outputBufferCapacity);
|
||||
_out.rewind(0, outputBufferCapacity);
|
||||
|
||||
LOG("[BatchHandler::onSetup] input Capacity %d, output capacity %d",
|
||||
inputBufferCapacity, _out.limit());
|
||||
}
|
||||
configure(_config);
|
||||
}
|
||||
|
||||
ResultBuffer * BatchHandler::call(const Command& cmd, ParameterBuffer * param) {
|
||||
JNIEnv * env = JNU_GetJNIEnv();
|
||||
jbyteArray jcmdData = JNU_ReadWriteBufferToByteArray(env, param);
|
||||
jbyteArray ret = (jbyteArray)env->CallObjectMethod((jobject)_processor, SendCommandToJavaMethodID,
|
||||
cmd.id(), jcmdData);
|
||||
|
||||
|
||||
if (env->ExceptionCheck()) {
|
||||
THROW_EXCEPTION(JavaException, "SendCommandToJava throw exception");
|
||||
}
|
||||
return JNU_ByteArraytoReadWriteBuffer(env, ret);
|
||||
}
|
||||
|
||||
} // namespace NativeTask
|
||||
|
||||
///////////////////////////////////////////////////////////////
|
||||
// NativeBatchProcessor jni methods
|
||||
///////////////////////////////////////////////////////////////
|
||||
using namespace NativeTask;
|
||||
|
||||
/*
|
||||
* Class: org_apache_hadoop_mapred_nativetask_NativeBatchProcessor
|
||||
* Method: setupHandler
|
||||
* Signature: (J)V
|
||||
*/
|
||||
void JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeBatchProcessor_setupHandler(
|
||||
JNIEnv * jenv, jobject processor, jlong handler, jobjectArray configs) {
|
||||
try {
|
||||
|
||||
NativeTask::Config * config = new NativeTask::Config();
|
||||
jsize len = jenv->GetArrayLength(configs);
|
||||
for (jsize i = 0; i + 1 < len; i += 2) {
|
||||
jbyteArray key_obj = (jbyteArray)jenv->GetObjectArrayElement(configs, i);
|
||||
jbyteArray val_obj = (jbyteArray)jenv->GetObjectArrayElement(configs, i + 1);
|
||||
config->set(JNU_ByteArrayToString(jenv, key_obj), JNU_ByteArrayToString(jenv, val_obj));
|
||||
}
|
||||
|
||||
NativeTask::BatchHandler * batchHandler = (NativeTask::BatchHandler *)((void*)handler);
|
||||
if (NULL == batchHandler) {
|
||||
JNU_ThrowByName(jenv, "java/lang/IllegalArgumentException", "BatchHandler is null");
|
||||
return;
|
||||
}
|
||||
jobject jinputBuffer = jenv->GetObjectField(processor, InputBufferFieldID);
|
||||
char * inputBufferAddr = NULL;
|
||||
uint32_t inputBufferCapacity = 0;
|
||||
if (NULL != jinputBuffer) {
|
||||
inputBufferAddr = (char*)(jenv->GetDirectBufferAddress(jinputBuffer));
|
||||
inputBufferCapacity = jenv->GetDirectBufferCapacity(jinputBuffer);
|
||||
}
|
||||
jobject joutputBuffer = jenv->GetObjectField(processor, OutputBufferFieldID);
|
||||
char * outputBufferAddr = NULL;
|
||||
uint32_t outputBufferCapacity = 0;
|
||||
if (NULL != joutputBuffer) {
|
||||
outputBufferAddr = (char*)(jenv->GetDirectBufferAddress(joutputBuffer));
|
||||
outputBufferCapacity = jenv->GetDirectBufferCapacity(joutputBuffer);
|
||||
}
|
||||
batchHandler->setProcessor(jenv->NewGlobalRef(processor));
|
||||
batchHandler->onSetup(config, inputBufferAddr, inputBufferCapacity, outputBufferAddr,
|
||||
outputBufferCapacity);
|
||||
} catch (NativeTask::UnsupportException & e) {
|
||||
JNU_ThrowByName(jenv, "java/lang/UnsupportedOperationException", e.what());
|
||||
} catch (NativeTask::OutOfMemoryException & e) {
|
||||
JNU_ThrowByName(jenv, "java/lang/OutOfMemoryError", e.what());
|
||||
} catch (NativeTask::IOException & e) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", e.what());
|
||||
} catch (NativeTask::JavaException & e) {
|
||||
LOG("JavaException: %s", e.what());
|
||||
// Do nothing, let java side handle
|
||||
} catch (std::exception & e) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", e.what());
|
||||
} catch (...) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", "Unknown exception");
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_apache_hadoop_mapred_nativetask_NativeBatchProcessor
|
||||
* Method: nativeProcessInput
|
||||
* Signature: (JI)V
|
||||
*/
|
||||
void JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeBatchProcessor_nativeProcessInput(
|
||||
JNIEnv * jenv, jobject processor, jlong handler, jint length) {
|
||||
|
||||
try {
|
||||
NativeTask::BatchHandler * batchHandler = (NativeTask::BatchHandler *)((void*)handler);
|
||||
if (NULL == batchHandler) {
|
||||
JNU_ThrowByName(jenv, "java/lang/IllegalArgumentException",
|
||||
"handler not instance of BatchHandler");
|
||||
return;
|
||||
}
|
||||
batchHandler->onInputData(length);
|
||||
} catch (NativeTask::UnsupportException & e) {
|
||||
JNU_ThrowByName(jenv, "java/lang/UnsupportedOperationException", e.what());
|
||||
} catch (NativeTask::OutOfMemoryException & e) {
|
||||
JNU_ThrowByName(jenv, "java/lang/OutOfMemoryError", e.what());
|
||||
} catch (NativeTask::IOException & e) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", e.what());
|
||||
} catch (NativeTask::JavaException & e) {
|
||||
LOG("JavaException: %s", e.what());
|
||||
// Do nothing, let java side handle
|
||||
} catch (std::exception & e) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", e.what());
|
||||
} catch (...) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", "Unknown exception");
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_apache_hadoop_mapred_nativetask_NativeBatchProcessor
|
||||
* Method: nativeFinish
|
||||
* Signature: (J)V
|
||||
*/
|
||||
void JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeBatchProcessor_nativeFinish(
|
||||
JNIEnv * jenv, jobject processor, jlong handler) {
|
||||
try {
|
||||
NativeTask::BatchHandler * batchHandler = (NativeTask::BatchHandler *)((void*)handler);
|
||||
if (NULL == batchHandler) {
|
||||
JNU_ThrowByName(jenv, "java/lang/IllegalArgumentException",
|
||||
"handler not instance of BatchHandler");
|
||||
return;
|
||||
}
|
||||
batchHandler->onFinish();
|
||||
} catch (NativeTask::UnsupportException & e) {
|
||||
JNU_ThrowByName(jenv, "java/lang/UnsupportedOperationException", e.what());
|
||||
} catch (NativeTask::OutOfMemoryException & e) {
|
||||
JNU_ThrowByName(jenv, "java/lang/OutOfMemoryError", e.what());
|
||||
} catch (NativeTask::IOException & e) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", e.what());
|
||||
} catch (NativeTask::JavaException & e) {
|
||||
LOG("JavaException: %s", e.what());
|
||||
// Do nothing, let java side handle
|
||||
} catch (std::exception & e) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", e.what());
|
||||
} catch (...) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", "Unknown exception");
|
||||
}
|
||||
}
|
||||
|
||||
void JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeBatchProcessor_nativeLoadData(
|
||||
JNIEnv * jenv, jobject processor, jlong handler) {
|
||||
try {
|
||||
NativeTask::BatchHandler * batchHandler = (NativeTask::BatchHandler *)((void*)handler);
|
||||
if (NULL == batchHandler) {
|
||||
JNU_ThrowByName(jenv, "java/lang/IllegalArgumentException",
|
||||
"handler not instance of BatchHandler");
|
||||
return;
|
||||
}
|
||||
batchHandler->onLoadData();
|
||||
} catch (NativeTask::UnsupportException & e) {
|
||||
JNU_ThrowByName(jenv, "java/lang/UnsupportedOperationException", e.what());
|
||||
} catch (NativeTask::OutOfMemoryException & e) {
|
||||
JNU_ThrowByName(jenv, "java/lang/OutOfMemoryError", e.what());
|
||||
} catch (NativeTask::IOException & e) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", e.what());
|
||||
} catch (NativeTask::JavaException & e) {
|
||||
LOG("JavaException: %s", e.what());
|
||||
// Do nothing, let java side handle
|
||||
} catch (std::exception & e) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", e.what());
|
||||
} catch (...) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", "Unknown exception");
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_apache_hadoop_mapred_nativetask_NativeBatchProcessor
|
||||
* Method: nativeCommand
|
||||
* Signature: (J[B)[B
|
||||
*/
|
||||
jbyteArray JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeBatchProcessor_nativeCommand(
|
||||
JNIEnv * jenv, jobject processor, jlong handler, jint command, jbyteArray cmdData) {
|
||||
try {
|
||||
NativeTask::BatchHandler * batchHandler = (NativeTask::BatchHandler *)((void*)handler);
|
||||
if (NULL == batchHandler) {
|
||||
JNU_ThrowByName(jenv, "java/lang/IllegalArgumentException",
|
||||
"handler not instance of BatchHandler");
|
||||
return NULL;
|
||||
}
|
||||
Command cmd(command);
|
||||
ParameterBuffer * param = JNU_ByteArraytoReadWriteBuffer(jenv, cmdData);
|
||||
ResultBuffer * result = batchHandler->onCall(cmd, param);
|
||||
jbyteArray ret = JNU_ReadWriteBufferToByteArray(jenv, result);
|
||||
|
||||
delete result;
|
||||
delete param;
|
||||
return ret;
|
||||
} catch (NativeTask::UnsupportException & e) {
|
||||
JNU_ThrowByName(jenv, "java/lang/UnsupportedOperationException", e.what());
|
||||
} catch (NativeTask::OutOfMemoryException & e) {
|
||||
JNU_ThrowByName(jenv, "java/lang/OutOfMemoryError", e.what());
|
||||
} catch (NativeTask::IOException & e) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", e.what());
|
||||
} catch (const NativeTask::JavaException & e) {
|
||||
LOG("JavaException: %s", e.what());
|
||||
// Do nothing, let java side handle
|
||||
} catch (std::exception & e) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", e.what());
|
||||
} catch (...) {
|
||||
JNU_ThrowByName(jenv, "java/io/IOException", "Unknown exception");
|
||||
}
|
||||
return NULL;
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_apace_hadoop_mapred_nativetask_NativeBatchProcessor
|
||||
* Method: InitIDs
|
||||
* Signature: ()V
|
||||
*/
|
||||
void JNICALL Java_org_apache_hadoop_mapred_nativetask_NativeBatchProcessor_InitIDs(JNIEnv * jenv,
|
||||
jclass processorClass) {
|
||||
InputBufferFieldID = jenv->GetFieldID(processorClass, "rawOutputBuffer", "Ljava/nio/ByteBuffer;");
|
||||
OutputBufferFieldID = jenv->GetFieldID(processorClass, "rawInputBuffer", "Ljava/nio/ByteBuffer;");
|
||||
FlushOutputMethodID = jenv->GetMethodID(processorClass, "flushOutput", "(I)V");
|
||||
FinishOutputMethodID = jenv->GetMethodID(processorClass, "finishOutput", "()V");
|
||||
SendCommandToJavaMethodID = jenv->GetMethodID(processorClass, "sendCommandToJava", "(I[B)[B");
|
||||
}
|
||||
|
|
@ -0,0 +1,157 @@
|
|||
/*
|
||||
* 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 BATCHHANDLER_H_
|
||||
#define BATCHHANDLER_H_
|
||||
|
||||
#include "NativeTask.h"
|
||||
#include "lib/Buffers.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
/**
|
||||
* Native side counterpart of java side NativeBatchProcessor
|
||||
*/
|
||||
class BatchHandler : public Configurable {
|
||||
protected:
|
||||
ByteBuffer _in;
|
||||
ByteBuffer _out;
|
||||
void * _processor;
|
||||
Config * _config;
|
||||
public:
|
||||
BatchHandler();
|
||||
virtual ~BatchHandler();
|
||||
|
||||
virtual NativeObjectType type() {
|
||||
return BatchHandlerType;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by native jni functions to set global jni reference
|
||||
*/
|
||||
void setProcessor(void * processor) {
|
||||
_processor = processor;
|
||||
}
|
||||
|
||||
void releaseProcessor();
|
||||
|
||||
/**
|
||||
* Called by java side to setup native side BatchHandler
|
||||
* initialize buffers by default
|
||||
*/
|
||||
void onSetup(Config * config, char * inputBuffer, uint32_t inputBufferCapacity,
|
||||
char * outputBuffer, uint32_t outputBufferCapacity);
|
||||
|
||||
/**
|
||||
* Called by java side to notice that input data available to handle
|
||||
* @param length input buffer's available data length
|
||||
*/
|
||||
void onInputData(uint32_t length);
|
||||
|
||||
virtual void onLoadData() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by java side to notice that input has finished
|
||||
*/
|
||||
void onFinish() {
|
||||
finish();
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by java side to send command to this handler
|
||||
* BatchHandler ignore all command by default
|
||||
* @param cmd command data
|
||||
* @return command return value
|
||||
*/
|
||||
virtual ResultBuffer * onCall(const Command& command, ReadWriteBuffer * param) {
|
||||
return NULL;
|
||||
}
|
||||
|
||||
protected:
|
||||
virtual ResultBuffer * call(const Command& cmd, ParameterBuffer * param);
|
||||
|
||||
/**
|
||||
* Used by subclass, call java side flushOutput(int length)
|
||||
* @param length output buffer's available data length
|
||||
*/
|
||||
virtual void flushOutput();
|
||||
|
||||
/**
|
||||
* Used by subclass, call java side finishOutput()
|
||||
*/
|
||||
void finishOutput();
|
||||
|
||||
/**
|
||||
* Write output buffer and use flushOutput manually,
|
||||
* or use this helper method
|
||||
*/
|
||||
inline void output(const char * buff, uint32_t length) {
|
||||
while (length > 0) {
|
||||
uint32_t remain = _out.remain();
|
||||
if (length > remain) {
|
||||
flushOutput();
|
||||
}
|
||||
uint32_t cp = length < remain ? length : remain;
|
||||
simple_memcpy(_out.current(), buff, cp);
|
||||
buff += cp;
|
||||
length -= cp;
|
||||
_out.advance(cp);
|
||||
}
|
||||
}
|
||||
|
||||
inline void outputInt(uint32_t v) {
|
||||
if (4 > _out.remain()) {
|
||||
flushOutput();
|
||||
}
|
||||
*(uint32_t*)(_out.current()) = v;
|
||||
_out.advance(4);
|
||||
}
|
||||
|
||||
/////////////////////////////////////////////////////////////
|
||||
// Subclass should implement these if needed
|
||||
/////////////////////////////////////////////////////////////
|
||||
|
||||
/**
|
||||
* Called by onSetup, do nothing by default
|
||||
* Subclass should override this if needed
|
||||
*/
|
||||
virtual void configure(Config * config) {
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by onFinish, flush & close output by default
|
||||
* Subclass should override this if needed
|
||||
*/
|
||||
virtual void finish() {
|
||||
flushOutput();
|
||||
finishOutput();
|
||||
}
|
||||
;
|
||||
|
||||
/**
|
||||
* Called by onInputData, internal input data processor,
|
||||
* Subclass should override this if needed
|
||||
*/
|
||||
virtual void handleInput(ByteBuffer & byteBuffer) {
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace NativeTask
|
||||
|
||||
#endif /* BATCHHANDLER_H_ */
|
|
@ -0,0 +1,254 @@
|
|||
/**
|
||||
* 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 "CombineHandler.h"
|
||||
|
||||
namespace NativeTask {
|
||||
const char * REFILL = "refill";
|
||||
const int LENGTH_OF_REFILL_STRING = 6;
|
||||
|
||||
const Command CombineHandler::COMBINE(4, "Combine");
|
||||
|
||||
CombineHandler::CombineHandler()
|
||||
: _combineContext(NULL), _kvIterator(NULL), _writer(NULL), _kType(UnknownType),
|
||||
_vType(UnknownType), _config(NULL), _kvCached(false), _combineInputRecordCount(0),
|
||||
_combineInputBytes(0), _combineOutputRecordCount(0), _combineOutputBytes(0) {
|
||||
}
|
||||
|
||||
CombineHandler::~CombineHandler() {
|
||||
}
|
||||
|
||||
void CombineHandler::configure(Config * config) {
|
||||
|
||||
_config = config;
|
||||
MapOutputSpec::getSpecFromConfig(_config, _mapOutputSpec);
|
||||
_kType = _mapOutputSpec.keyType;
|
||||
_vType = _mapOutputSpec.valueType;
|
||||
}
|
||||
|
||||
uint32_t CombineHandler::feedDataToJavaInWritableSerialization() {
|
||||
|
||||
uint32_t written = 0;
|
||||
bool firstKV = true;
|
||||
_out.position(0);
|
||||
|
||||
if (_kvCached) {
|
||||
uint32_t kvLength = _key.outerLength + _value.outerLength + KVBuffer::headerLength();
|
||||
outputInt(bswap(_key.outerLength));
|
||||
outputInt(bswap(_value.outerLength));
|
||||
outputKeyOrValue(_key, _kType);
|
||||
outputKeyOrValue(_value, _vType);
|
||||
|
||||
written += kvLength;
|
||||
_kvCached = false;
|
||||
firstKV = false;
|
||||
}
|
||||
|
||||
uint32_t recordCount = 0;
|
||||
while (nextKeyValue(_key, _value)) {
|
||||
|
||||
//::sleep(5);
|
||||
_kvCached = false;
|
||||
recordCount++;
|
||||
|
||||
uint32_t kvLength = _key.outerLength + _value.outerLength + KVBuffer::headerLength();
|
||||
|
||||
if (!firstKV && kvLength > _out.remain()) {
|
||||
_kvCached = true;
|
||||
break;
|
||||
} else {
|
||||
firstKV = false;
|
||||
//write final key length and final value length
|
||||
outputInt(bswap(_key.outerLength));
|
||||
outputInt(bswap(_value.outerLength));
|
||||
outputKeyOrValue(_key, _kType);
|
||||
outputKeyOrValue(_value, _vType);
|
||||
|
||||
written += kvLength;
|
||||
}
|
||||
}
|
||||
|
||||
if (_out.position() > 0) {
|
||||
flushOutput();
|
||||
}
|
||||
|
||||
_combineInputRecordCount += recordCount;
|
||||
_combineInputBytes += written;
|
||||
return written;
|
||||
}
|
||||
|
||||
/**
|
||||
* KV: key or value
|
||||
*/
|
||||
void CombineHandler::outputKeyOrValue(SerializeInfo & KV, KeyValueType type) {
|
||||
switch (type) {
|
||||
case TextType:
|
||||
output(KV.varBytes, KV.outerLength - KV.buffer.length());
|
||||
output(KV.buffer.data(), KV.buffer.length());
|
||||
break;
|
||||
case BytesType:
|
||||
outputInt(bswap(KV.buffer.length()));
|
||||
output(KV.buffer.data(), KV.buffer.length());
|
||||
break;
|
||||
default:
|
||||
output(KV.buffer.data(), KV.buffer.length());
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
bool CombineHandler::nextKeyValue(SerializeInfo & key, SerializeInfo & value) {
|
||||
|
||||
if (!_kvIterator->next(key.buffer, value.buffer)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
uint32_t varLength = 0;
|
||||
switch (_kType) {
|
||||
case TextType:
|
||||
WritableUtils::WriteVInt(key.buffer.length(), key.varBytes, varLength);
|
||||
key.outerLength = key.buffer.length() + varLength;
|
||||
break;
|
||||
case BytesType:
|
||||
key.outerLength = key.buffer.length() + 4;
|
||||
break;
|
||||
default:
|
||||
key.outerLength = key.buffer.length();
|
||||
break;
|
||||
}
|
||||
|
||||
//prepare final value length
|
||||
uint32_t varValueLength = 0;
|
||||
switch (_vType) {
|
||||
case TextType:
|
||||
WritableUtils::WriteVInt(value.buffer.length(), value.varBytes, varValueLength);
|
||||
value.outerLength = value.buffer.length() + varValueLength;
|
||||
break;
|
||||
case BytesType:
|
||||
value.outerLength = value.buffer.length() + 4;
|
||||
break;
|
||||
default:
|
||||
value.outerLength = value.buffer.length();
|
||||
break;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
uint32_t CombineHandler::feedDataToJava(SerializationFramework serializationType) {
|
||||
if (serializationType == WRITABLE_SERIALIZATION) {
|
||||
return feedDataToJavaInWritableSerialization();
|
||||
}
|
||||
THROW_EXCEPTION(IOException, "Native Serialization not supported");
|
||||
}
|
||||
|
||||
void CombineHandler::handleInput(ByteBuffer & in) {
|
||||
char * buff = in.current();
|
||||
uint32_t length = in.remain();
|
||||
uint32_t remain = length;
|
||||
char * pos = buff;
|
||||
if (_asideBuffer.remain() > 0) {
|
||||
uint32_t filledLength = _asideBuffer.fill(pos, length);
|
||||
pos += filledLength;
|
||||
remain -= filledLength;
|
||||
}
|
||||
|
||||
if (_asideBuffer.size() > 0 && _asideBuffer.remain() == 0) {
|
||||
_asideBuffer.position(0);
|
||||
write(_asideBuffer.current(), _asideBuffer.size());
|
||||
_asideBuffer.wrap(NULL, 0);
|
||||
}
|
||||
|
||||
if (remain == 0) {
|
||||
return;
|
||||
}
|
||||
KVBuffer * kvBuffer = (KVBuffer *)pos;
|
||||
|
||||
if (unlikely(remain < kvBuffer->headerLength())) {
|
||||
THROW_EXCEPTION(IOException, "k/v meta information incomplete");
|
||||
}
|
||||
|
||||
uint32_t kvLength = kvBuffer->lengthConvertEndium();
|
||||
|
||||
if (kvLength > remain) {
|
||||
_asideBytes.resize(kvLength);
|
||||
_asideBuffer.wrap(_asideBytes.buff(), _asideBytes.size());
|
||||
_asideBuffer.fill(pos, remain);
|
||||
pos += remain;
|
||||
remain = 0;
|
||||
} else {
|
||||
write(pos, remain);
|
||||
}
|
||||
}
|
||||
|
||||
void CombineHandler::write(char * buf, uint32_t length) {
|
||||
KVBuffer * kv = NULL;
|
||||
char * pos = buf;
|
||||
uint32_t remain = length;
|
||||
|
||||
uint32_t outputRecordCount = 0;
|
||||
while (remain > 0) {
|
||||
kv = (KVBuffer *)pos;
|
||||
kv->keyLength = bswap(kv->keyLength);
|
||||
kv->valueLength = bswap(kv->valueLength);
|
||||
_writer->write(kv->getKey(), kv->keyLength, kv->getValue(), kv->valueLength);
|
||||
outputRecordCount++;
|
||||
remain -= kv->length();
|
||||
pos += kv->length();
|
||||
}
|
||||
|
||||
_combineOutputRecordCount += outputRecordCount;
|
||||
_combineOutputBytes += length;
|
||||
}
|
||||
|
||||
string toString(uint32_t length) {
|
||||
string result;
|
||||
result.reserve(4);
|
||||
result.assign((char *)(&length), 4);
|
||||
return result;
|
||||
}
|
||||
|
||||
void CombineHandler::onLoadData() {
|
||||
feedDataToJava(WRITABLE_SERIALIZATION);
|
||||
}
|
||||
|
||||
ResultBuffer * CombineHandler::onCall(const Command& command, ParameterBuffer * param) {
|
||||
THROW_EXCEPTION(UnsupportException, "Command not supported by RReducerHandler");
|
||||
}
|
||||
|
||||
void CombineHandler::combine(CombineContext type, KVIterator * kvIterator, IFileWriter * writer) {
|
||||
|
||||
_combineInputRecordCount = 0;
|
||||
_combineOutputRecordCount = 0;
|
||||
_combineInputBytes = 0;
|
||||
_combineOutputBytes = 0;
|
||||
|
||||
this->_combineContext = &type;
|
||||
this->_kvIterator = kvIterator;
|
||||
this->_writer = writer;
|
||||
call(COMBINE, NULL);
|
||||
|
||||
LOG("[CombineHandler] input Record Count: %d, input Bytes: %d, "
|
||||
"output Record Count: %d, output Bytes: %d",
|
||||
_combineInputRecordCount, _combineInputBytes,
|
||||
_combineOutputRecordCount, _combineOutputBytes);
|
||||
return;
|
||||
}
|
||||
|
||||
void CombineHandler::finish() {
|
||||
}
|
||||
|
||||
} /* namespace NativeTask */
|
|
@ -0,0 +1,90 @@
|
|||
/**
|
||||
* 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 _COMBINEHANDLER_H_
|
||||
#define _COMBINEHANDLER_H_
|
||||
|
||||
#include "lib/Combiner.h"
|
||||
#include "BatchHandler.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
enum SerializationFramework {
|
||||
WRITABLE_SERIALIZATION = 0,
|
||||
NATIVE_SERIALIZATION = 1
|
||||
};
|
||||
|
||||
struct SerializeInfo {
|
||||
Buffer buffer;
|
||||
uint32_t outerLength;
|
||||
char varBytes[8];
|
||||
};
|
||||
|
||||
class CombineHandler : public NativeTask::ICombineRunner, public NativeTask::BatchHandler {
|
||||
public:
|
||||
static const Command COMBINE;
|
||||
|
||||
private:
|
||||
|
||||
CombineContext * _combineContext;
|
||||
KVIterator * _kvIterator;
|
||||
IFileWriter * _writer;
|
||||
SerializeInfo _key;
|
||||
SerializeInfo _value;
|
||||
|
||||
KeyValueType _kType;
|
||||
KeyValueType _vType;
|
||||
MapOutputSpec _mapOutputSpec;
|
||||
Config * _config;
|
||||
bool _kvCached;
|
||||
|
||||
uint32_t _combineInputRecordCount;
|
||||
uint32_t _combineInputBytes;
|
||||
|
||||
uint32_t _combineOutputRecordCount;
|
||||
uint32_t _combineOutputBytes;
|
||||
|
||||
FixSizeContainer _asideBuffer;
|
||||
ByteArray _asideBytes;
|
||||
|
||||
public:
|
||||
CombineHandler();
|
||||
virtual ~CombineHandler();
|
||||
|
||||
virtual void handleInput(ByteBuffer & byteBuffer);
|
||||
void finish();
|
||||
|
||||
ResultBuffer * onCall(const Command& command, ParameterBuffer * param);
|
||||
|
||||
void configure(Config * config);
|
||||
|
||||
void combine(CombineContext type, KVIterator * kvIterator, IFileWriter * writer);
|
||||
|
||||
virtual void onLoadData();
|
||||
|
||||
private:
|
||||
void flushDataToWriter();
|
||||
void outputKeyOrValue(SerializeInfo & info, KeyValueType type);
|
||||
bool nextKeyValue(SerializeInfo & key, SerializeInfo & value);
|
||||
uint32_t feedDataToJava(SerializationFramework serializationType);
|
||||
uint32_t feedDataToJavaInWritableSerialization();
|
||||
void write(char * buf, uint32_t length);
|
||||
|
||||
};
|
||||
|
||||
} /* namespace NativeTask */
|
||||
#endif /* _JAVACOMBINEHANDLER_H_ */
|
|
@ -0,0 +1,98 @@
|
|||
/*
|
||||
* 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 "lib/commons.h"
|
||||
#include "util/StringUtil.h"
|
||||
#include "lib/TaskCounters.h"
|
||||
#include "MCollectorOutputHandler.h"
|
||||
#include "lib/NativeObjectFactory.h"
|
||||
#include "lib/MapOutputCollector.h"
|
||||
#include "CombineHandler.h"
|
||||
|
||||
using std::string;
|
||||
using std::vector;
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
MCollectorOutputHandler::MCollectorOutputHandler()
|
||||
: _collector(NULL), _dest(NULL), _endium(LARGE_ENDIUM) {
|
||||
}
|
||||
|
||||
MCollectorOutputHandler::~MCollectorOutputHandler() {
|
||||
_dest = NULL;
|
||||
delete _collector;
|
||||
_collector = NULL;
|
||||
}
|
||||
|
||||
void MCollectorOutputHandler::configure(Config * config) {
|
||||
if (NULL == config) {
|
||||
return;
|
||||
}
|
||||
|
||||
uint32_t partition = config->getInt(MAPRED_NUM_REDUCES, 1);
|
||||
|
||||
_collector = new MapOutputCollector(partition, this);
|
||||
_collector->configure(config);
|
||||
}
|
||||
|
||||
void MCollectorOutputHandler::finish() {
|
||||
_collector->close();
|
||||
BatchHandler::finish();
|
||||
}
|
||||
|
||||
void MCollectorOutputHandler::handleInput(ByteBuffer & in) {
|
||||
char * buff = in.current();
|
||||
uint32_t length = in.remain();
|
||||
|
||||
const char * end = buff + length;
|
||||
char * pos = buff;
|
||||
if (_kvContainer.remain() > 0) {
|
||||
uint32_t filledLength = _kvContainer.fill(pos, length);
|
||||
pos += filledLength;
|
||||
}
|
||||
|
||||
while (end - pos > 0) {
|
||||
KVBufferWithParititionId * kvBuffer = (KVBufferWithParititionId *)pos;
|
||||
|
||||
if (unlikely(end - pos < KVBuffer::headerLength())) {
|
||||
THROW_EXCEPTION(IOException, "k/v meta information incomplete");
|
||||
}
|
||||
|
||||
if (_endium == LARGE_ENDIUM) {
|
||||
kvBuffer->partitionId = bswap(kvBuffer->partitionId);
|
||||
kvBuffer->buffer.keyLength = bswap(kvBuffer->buffer.keyLength);
|
||||
kvBuffer->buffer.valueLength = bswap(kvBuffer->buffer.valueLength);
|
||||
}
|
||||
|
||||
uint32_t kvLength = kvBuffer->buffer.length();
|
||||
|
||||
KVBuffer * dest = allocateKVBuffer(kvBuffer->partitionId, kvLength);
|
||||
_kvContainer.wrap((char *)dest, kvLength);
|
||||
|
||||
pos += 4; //skip the partition length
|
||||
uint32_t filledLength = _kvContainer.fill(pos, end - pos);
|
||||
pos += filledLength;
|
||||
}
|
||||
}
|
||||
|
||||
KVBuffer * MCollectorOutputHandler::allocateKVBuffer(uint32_t partitionId, uint32_t kvlength) {
|
||||
KVBuffer * dest = _collector->allocateKVBuffer(partitionId, kvlength);
|
||||
return dest;
|
||||
}
|
||||
|
||||
} // namespace NativeTask
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
* 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 MCOLLECTOROUTPUTHANDLER_H_
|
||||
#define MCOLLECTOROUTPUTHANDLER_H_
|
||||
|
||||
#include "BatchHandler.h"
|
||||
#include "lib/SpillOutputService.h"
|
||||
#include "AbstractMapHandler.h"
|
||||
|
||||
namespace NativeTask {
|
||||
class MapOutputCollector;
|
||||
|
||||
class MCollectorOutputHandler : public AbstractMapHandler {
|
||||
private:
|
||||
|
||||
FixSizeContainer _kvContainer;
|
||||
|
||||
MapOutputCollector * _collector;
|
||||
// state info for large KV pairs
|
||||
char * _dest;
|
||||
|
||||
Endium _endium;
|
||||
|
||||
public:
|
||||
MCollectorOutputHandler();
|
||||
virtual ~MCollectorOutputHandler();
|
||||
|
||||
virtual void configure(Config * config);
|
||||
virtual void finish();
|
||||
virtual void handleInput(ByteBuffer & byteBuffer);
|
||||
private:
|
||||
KVBuffer * allocateKVBuffer(uint32_t partition, uint32_t kvlength);
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif /* MCOLLECTOROUTPUTHANDLER_H_ */
|
|
@ -0,0 +1,43 @@
|
|||
/**
|
||||
* 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 "lib/commons.h"
|
||||
#include "lib/BufferStream.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
int32_t InputBuffer::read(void * buff, uint32_t length) {
|
||||
uint32_t rd = _capacity - _position < length ? _capacity - _position : length;
|
||||
if (rd > 0) {
|
||||
memcpy(buff, _buff + _position, rd);
|
||||
_position += rd;
|
||||
return rd;
|
||||
}
|
||||
return length == 0 ? 0 : -1;
|
||||
}
|
||||
|
||||
void OutputBuffer::write(const void * buff, uint32_t length) {
|
||||
if (_position + length <= _capacity) {
|
||||
memcpy(_buff + _position, buff, length);
|
||||
_position += length;
|
||||
} else {
|
||||
THROW_EXCEPTION(IOException, "OutputBuffer too small to write");
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace NativeTask
|
|
@ -0,0 +1,156 @@
|
|||
/**
|
||||
* 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 BUFFERSTREAM_H_
|
||||
#define BUFFERSTREAM_H_
|
||||
|
||||
#include <string>
|
||||
#include "lib/Streams.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
using std::string;
|
||||
|
||||
class InputBuffer : public InputStream {
|
||||
protected:
|
||||
const char * _buff;
|
||||
uint32_t _position;
|
||||
uint32_t _capacity;
|
||||
public:
|
||||
InputBuffer()
|
||||
: _buff(NULL), _position(0), _capacity(0) {
|
||||
}
|
||||
|
||||
InputBuffer(const char * buff, uint32_t capacity)
|
||||
: _buff(buff), _position(0), _capacity(capacity) {
|
||||
}
|
||||
|
||||
InputBuffer(const string & src)
|
||||
: _buff(src.data()), _position(0), _capacity(src.length()) {
|
||||
}
|
||||
|
||||
virtual ~InputBuffer() {
|
||||
}
|
||||
|
||||
virtual void seek(uint64_t position) {
|
||||
if (position <= _capacity) {
|
||||
_position = position;
|
||||
} else {
|
||||
_position = _capacity;
|
||||
}
|
||||
}
|
||||
|
||||
virtual uint64_t tell() {
|
||||
return _position;
|
||||
}
|
||||
|
||||
virtual int32_t read(void * buff, uint32_t length);
|
||||
|
||||
void reset(const char * buff, uint32_t capacity) {
|
||||
_buff = buff;
|
||||
_position = 0;
|
||||
_capacity = capacity;
|
||||
}
|
||||
|
||||
void reset(const string & src) {
|
||||
_buff = src.data();
|
||||
_position = 0;
|
||||
_capacity = src.length();
|
||||
}
|
||||
|
||||
void rewind() {
|
||||
_position = 0;
|
||||
}
|
||||
};
|
||||
|
||||
class OutputBuffer : public OutputStream {
|
||||
protected:
|
||||
char * _buff;
|
||||
uint32_t _position;
|
||||
uint32_t _capacity;
|
||||
public:
|
||||
OutputBuffer()
|
||||
: _buff(NULL), _position(0), _capacity(0) {
|
||||
}
|
||||
|
||||
OutputBuffer(char * buff, uint32_t capacity)
|
||||
: _buff(buff), _position(0), _capacity(capacity) {
|
||||
}
|
||||
|
||||
virtual ~OutputBuffer() {
|
||||
}
|
||||
|
||||
virtual uint64_t tell() {
|
||||
return _position;
|
||||
}
|
||||
|
||||
virtual void write(const void * buff, uint32_t length);
|
||||
|
||||
void clear() {
|
||||
_position = 0;
|
||||
}
|
||||
|
||||
void reset(char * buff, uint32_t capacity) {
|
||||
_buff = buff;
|
||||
_position = 0;
|
||||
_capacity = capacity;
|
||||
}
|
||||
|
||||
string getString() {
|
||||
return string(_buff, _position);
|
||||
}
|
||||
};
|
||||
|
||||
class OutputStringStream : public OutputStream {
|
||||
protected:
|
||||
string * _dest;
|
||||
public:
|
||||
OutputStringStream()
|
||||
: _dest(NULL) {
|
||||
}
|
||||
|
||||
OutputStringStream(string & dest)
|
||||
: _dest(&dest) {
|
||||
}
|
||||
virtual ~OutputStringStream() {
|
||||
}
|
||||
|
||||
virtual uint64_t tell() {
|
||||
return _dest->length();
|
||||
}
|
||||
|
||||
virtual void write(const void * buff, uint32_t length) {
|
||||
_dest->append((const char *)buff, length);
|
||||
}
|
||||
|
||||
void reset(string * dest) {
|
||||
_dest = dest;
|
||||
}
|
||||
|
||||
void clear() {
|
||||
_dest->clear();
|
||||
}
|
||||
|
||||
string getString() {
|
||||
return *_dest;
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace NativeTask
|
||||
|
||||
#endif /* BUFFERSTREAM_H_ */
|
|
@ -0,0 +1,235 @@
|
|||
/**
|
||||
* 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 <string>
|
||||
|
||||
#include "lib/commons.h"
|
||||
#include "util/StringUtil.h"
|
||||
#include "util/WritableUtils.h"
|
||||
#include "lib/Buffers.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
|
||||
ReadBuffer::ReadBuffer()
|
||||
: _buff(NULL), _remain(0), _size(0), _capacity(0), _stream(NULL), _source(NULL) {
|
||||
}
|
||||
|
||||
void ReadBuffer::init(uint32_t size, InputStream * stream, const string & codec) {
|
||||
if (size < 1024) {
|
||||
THROW_EXCEPTION_EX(UnsupportException, "ReadBuffer size %u not support.", size);
|
||||
}
|
||||
_buff = (char *)malloc(size);
|
||||
if (NULL == _buff) {
|
||||
THROW_EXCEPTION(OutOfMemoryException, "create append buffer");
|
||||
}
|
||||
_capacity = size;
|
||||
_remain = 0;
|
||||
_size = 0;
|
||||
_stream = stream;
|
||||
_source = _stream;
|
||||
if (codec.length() > 0) {
|
||||
if (!Compressions::support(codec)) {
|
||||
THROW_EXCEPTION(UnsupportException, "compression codec not support");
|
||||
}
|
||||
_source = Compressions::getDecompressionStream(codec, _stream, size);
|
||||
}
|
||||
}
|
||||
|
||||
ReadBuffer::~ReadBuffer() {
|
||||
if (_source != _stream) {
|
||||
delete _source;
|
||||
_source = NULL;
|
||||
}
|
||||
if (NULL != _buff) {
|
||||
free(_buff);
|
||||
_buff = NULL;
|
||||
_capacity = 0;
|
||||
_remain = 0;
|
||||
_size = 0;
|
||||
}
|
||||
}
|
||||
|
||||
char * ReadBuffer::fillGet(uint32_t count) {
|
||||
|
||||
if (unlikely(count > _capacity)) {
|
||||
uint32_t newcap = _capacity * 2 > count ? _capacity * 2 : count;
|
||||
char * newbuff = (char*)malloc(newcap);
|
||||
|
||||
|
||||
if (newbuff == NULL) {
|
||||
THROW_EXCEPTION(OutOfMemoryException,
|
||||
StringUtil::Format("buff realloc failed, size=%u", newcap));
|
||||
}
|
||||
|
||||
if (_remain > 0) {
|
||||
memcpy(newbuff, current(), _remain);
|
||||
}
|
||||
if (NULL != _buff) {
|
||||
free(_buff);
|
||||
}
|
||||
|
||||
_buff = newbuff;
|
||||
_capacity = newcap;
|
||||
} else {
|
||||
if (_remain > 0) {
|
||||
memmove(_buff, current(), _remain);
|
||||
}
|
||||
}
|
||||
_size = _remain;
|
||||
while (_remain < count) {
|
||||
int32_t rd = _source->read(_buff + _size, _capacity - _size);
|
||||
if (rd <= 0) {
|
||||
THROW_EXCEPTION(IOException, "read reach EOF");
|
||||
}
|
||||
_remain += rd;
|
||||
_size += rd;
|
||||
}
|
||||
char * ret = current();
|
||||
_remain -= count;
|
||||
return ret;
|
||||
}
|
||||
|
||||
int32_t ReadBuffer::fillRead(char * buff, uint32_t len) {
|
||||
uint32_t cp = _remain;
|
||||
if (cp > 0) {
|
||||
memcpy(buff, current(), cp);
|
||||
_remain = 0;
|
||||
}
|
||||
// TODO: read to buffer first
|
||||
int32_t ret = _source->readFully(buff + cp, len - cp);
|
||||
if (ret < 0 && cp == 0) {
|
||||
return ret;
|
||||
} else {
|
||||
return ret < 0 ? cp : ret + cp;
|
||||
}
|
||||
}
|
||||
|
||||
int64_t ReadBuffer::fillReadVLong() {
|
||||
if (_remain == 0) {
|
||||
int32_t rd = _source->read(_buff, _capacity);
|
||||
if (rd <= 0) {
|
||||
THROW_EXCEPTION(IOException, "fillReadVLong reach EOF");
|
||||
}
|
||||
_remain = rd;
|
||||
_size = rd;
|
||||
}
|
||||
int8_t * pos = (int8_t*)current();
|
||||
if (*pos >= -112) {
|
||||
_remain--;
|
||||
return (int64_t)*pos;
|
||||
}
|
||||
bool neg = *pos < -120;
|
||||
uint32_t len = neg ? (-119 - *pos) : (-111 - *pos);
|
||||
pos = (int8_t*)get(len);
|
||||
const int8_t * end = pos + len;
|
||||
uint64_t value = 0;
|
||||
while (++pos < end) {
|
||||
value = (value << 8) | *(uint8_t*)pos;
|
||||
}
|
||||
return neg ? (value ^ -1LL) : value;
|
||||
}
|
||||
|
||||
///////////////////////////////////////////////////////////
|
||||
|
||||
AppendBuffer::AppendBuffer()
|
||||
: _buff(NULL), _remain(0), _capacity(0), _counter(0), _stream(NULL), _dest(NULL),
|
||||
_compression(false) {
|
||||
}
|
||||
|
||||
void AppendBuffer::init(uint32_t size, OutputStream * stream, const string & codec) {
|
||||
if (size < 1024) {
|
||||
THROW_EXCEPTION_EX(UnsupportException, "AppendBuffer size %u not support.", size);
|
||||
}
|
||||
_buff = (char *)malloc(size + 8);
|
||||
if (NULL == _buff) {
|
||||
THROW_EXCEPTION(OutOfMemoryException, "create append buffer");
|
||||
}
|
||||
_capacity = size;
|
||||
_remain = _capacity;
|
||||
_stream = stream;
|
||||
_dest = _stream;
|
||||
if (codec.length() > 0) {
|
||||
if (!Compressions::support(codec)) {
|
||||
THROW_EXCEPTION(UnsupportException, "compression codec not support");
|
||||
}
|
||||
_dest = Compressions::getCompressionStream(codec, _stream, size);
|
||||
_compression = true;
|
||||
}
|
||||
}
|
||||
|
||||
CompressStream * AppendBuffer::getCompressionStream() {
|
||||
if (_compression) {
|
||||
return (CompressStream *)_dest;
|
||||
} else {
|
||||
return NULL;
|
||||
}
|
||||
}
|
||||
|
||||
AppendBuffer::~AppendBuffer() {
|
||||
if (_dest != _stream) {
|
||||
delete _dest;
|
||||
_dest = NULL;
|
||||
}
|
||||
if (NULL != _buff) {
|
||||
free(_buff);
|
||||
_buff = NULL;
|
||||
_remain = 0;
|
||||
_capacity = 0;
|
||||
}
|
||||
}
|
||||
|
||||
void AppendBuffer::flushd() {
|
||||
_dest->write(_buff, _capacity - _remain);
|
||||
_counter += _capacity - _remain;
|
||||
_remain = _capacity;
|
||||
}
|
||||
|
||||
void AppendBuffer::write_inner(const void * data, uint32_t len) {
|
||||
flushd();
|
||||
if (len >= _capacity / 2) {
|
||||
_dest->write(data, len);
|
||||
_counter += len;
|
||||
} else {
|
||||
simple_memcpy(_buff, data, len);
|
||||
_remain -= len;
|
||||
}
|
||||
}
|
||||
|
||||
void AppendBuffer::write_vlong_inner(int64_t v) {
|
||||
if (_remain < 9) {
|
||||
flushd();
|
||||
}
|
||||
uint32_t len;
|
||||
WritableUtils::WriteVLong(v, current(), len);
|
||||
_remain -= len;
|
||||
}
|
||||
|
||||
void AppendBuffer::write_vuint2_inner(uint32_t v1, uint32_t v2) {
|
||||
if (_remain < 10) {
|
||||
flushd();
|
||||
}
|
||||
uint32_t len;
|
||||
WritableUtils::WriteVLong(v1, current(), len);
|
||||
_remain -= len;
|
||||
WritableUtils::WriteVLong(v2, current(), len);
|
||||
_remain -= len;
|
||||
}
|
||||
|
||||
} // namespace NativeTask
|
||||
|
|
@ -0,0 +1,637 @@
|
|||
/**
|
||||
* 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 BUFFERS_H_
|
||||
#define BUFFERS_H_
|
||||
|
||||
#include "lib/Streams.h"
|
||||
#include "lib/Compressions.h"
|
||||
#include "lib/Constants.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
|
||||
/**
|
||||
* A lightweight read buffer, act as buffered input stream
|
||||
*/
|
||||
class ReadBuffer {
|
||||
protected:
|
||||
char * _buff;
|
||||
uint32_t _remain;
|
||||
uint32_t _size;
|
||||
uint32_t _capacity;
|
||||
|
||||
InputStream * _stream;
|
||||
InputStream * _source;
|
||||
|
||||
protected:
|
||||
inline char * current() {
|
||||
return _buff + _size - _remain;
|
||||
}
|
||||
|
||||
char * fillGet(uint32_t count);
|
||||
int32_t fillRead(char * buff, uint32_t len);
|
||||
int64_t fillReadVLong();
|
||||
public:
|
||||
ReadBuffer();
|
||||
|
||||
void init(uint32_t size, InputStream * stream, const string & codec);
|
||||
|
||||
~ReadBuffer();
|
||||
|
||||
/**
|
||||
* use get() to get inplace continuous memory of small object
|
||||
*/
|
||||
inline char * get(uint32_t count) {
|
||||
if (likely(count <= _remain)) {
|
||||
char * ret = current();
|
||||
_remain -= count;
|
||||
return ret;
|
||||
}
|
||||
return fillGet(count);
|
||||
}
|
||||
|
||||
/**
|
||||
* read to outside buffer
|
||||
*/
|
||||
inline int32_t read(char * buff, uint32_t len) {
|
||||
if (likely(len <= _remain)) {
|
||||
memcpy(buff, current(), len);
|
||||
_remain -= len;
|
||||
return len;
|
||||
}
|
||||
return fillRead(buff, len);
|
||||
}
|
||||
|
||||
/**
|
||||
* read to outside buffer, use simple_memcpy
|
||||
*/
|
||||
inline void readUnsafe(char * buff, uint32_t len) {
|
||||
if (likely(len <= _remain)) {
|
||||
simple_memcpy(buff, current(), len);
|
||||
_remain -= len;
|
||||
return;
|
||||
}
|
||||
fillRead(buff, len);
|
||||
}
|
||||
|
||||
/**
|
||||
* read VUInt
|
||||
*/
|
||||
inline int64_t readVLong() {
|
||||
if (likely(_remain > 0)) {
|
||||
char * mark = current();
|
||||
if (*(int8_t*)mark >= (int8_t)-112) {
|
||||
_remain--;
|
||||
return (int64_t)*mark;
|
||||
}
|
||||
}
|
||||
return fillReadVLong();
|
||||
}
|
||||
|
||||
/**
|
||||
* read uint32_t little endian
|
||||
*/
|
||||
inline uint32_t read_uint32_le() {
|
||||
return *(uint32_t*)get(4);
|
||||
}
|
||||
|
||||
/**
|
||||
* read uint32_t big endian
|
||||
*/
|
||||
inline uint32_t read_uint32_be() {
|
||||
return bswap(read_uint32_le());
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* A light weighted append buffer, used as buffered output streams
|
||||
*/
|
||||
class AppendBuffer {
|
||||
protected:
|
||||
char * _buff;
|
||||
uint32_t _remain;
|
||||
uint32_t _capacity;
|
||||
uint64_t _counter;
|
||||
|
||||
OutputStream * _stream;
|
||||
OutputStream * _dest;
|
||||
bool _compression;
|
||||
|
||||
protected:
|
||||
void flushd();
|
||||
|
||||
inline char * current() {
|
||||
return _buff + _capacity - _remain;
|
||||
}
|
||||
|
||||
void write_inner(const void * data, uint32_t len);
|
||||
void write_vlong_inner(int64_t v);
|
||||
void write_vuint2_inner(uint32_t v1, uint32_t v2);
|
||||
public:
|
||||
AppendBuffer();
|
||||
|
||||
~AppendBuffer();
|
||||
|
||||
void init(uint32_t size, OutputStream * stream, const string & codec);
|
||||
|
||||
CompressStream * getCompressionStream();
|
||||
|
||||
uint64_t getCounter() {
|
||||
return _counter;
|
||||
}
|
||||
|
||||
inline char * borrowUnsafe(uint32_t len) {
|
||||
if (likely(_remain >= len)) {
|
||||
return current();
|
||||
}
|
||||
if (likely(_capacity >= len)) {
|
||||
flushd();
|
||||
return _buff;
|
||||
}
|
||||
return NULL;
|
||||
}
|
||||
|
||||
inline void useUnsafe(uint32_t count) {
|
||||
_remain -= count;
|
||||
}
|
||||
|
||||
inline void write(char c) {
|
||||
if (unlikely(_remain == 0)) {
|
||||
flushd();
|
||||
}
|
||||
*current() = c;
|
||||
_remain--;
|
||||
}
|
||||
|
||||
inline void write(const void * data, uint32_t len) {
|
||||
if (likely(len <= _remain)) { // append directly
|
||||
simple_memcpy(current(), data, len);
|
||||
_remain -= len;
|
||||
return;
|
||||
}
|
||||
write_inner(data, len);
|
||||
}
|
||||
|
||||
inline void write_uint32_le(uint32_t v) {
|
||||
if (unlikely(4 > _remain)) {
|
||||
flushd();
|
||||
}
|
||||
*(uint32_t*)current() = v;
|
||||
_remain -= 4;
|
||||
return;
|
||||
}
|
||||
|
||||
inline void write_uint32_be(uint32_t v) {
|
||||
write_uint32_le(bswap(v));
|
||||
}
|
||||
|
||||
inline void write_uint64_le(uint64_t v) {
|
||||
if (unlikely(8 > _remain)) {
|
||||
flushd();
|
||||
}
|
||||
*(uint64_t*)current() = v;
|
||||
_remain -= 8;
|
||||
return;
|
||||
}
|
||||
|
||||
inline void write_uint64_be(uint64_t v) {
|
||||
write_uint64_le(bswap64(v));
|
||||
}
|
||||
|
||||
inline void write_vlong(int64_t v) {
|
||||
if (likely(_remain > 0 && v <= 127 && v >= -112)) {
|
||||
*(char*)current() = (char)v;
|
||||
_remain--;
|
||||
return;
|
||||
}
|
||||
write_vlong_inner(v);
|
||||
}
|
||||
|
||||
inline void write_vuint(uint32_t v) {
|
||||
if (likely(_remain > 0 && v <= 127)) {
|
||||
*(char*)current() = (char)v;
|
||||
_remain--;
|
||||
return;
|
||||
}
|
||||
write_vlong_inner(v);
|
||||
}
|
||||
|
||||
inline void write_vuint2(uint32_t v1, uint32_t v2) {
|
||||
if (likely(_remain >= 2 && v1 <= 127 && v2 <= 127)) {
|
||||
*(char*)current() = (char)v1;
|
||||
*(char*)(current() + 1) = (char)v2;
|
||||
_remain -= 2;
|
||||
return;
|
||||
}
|
||||
write_vuint2_inner(v1, v2);
|
||||
}
|
||||
|
||||
/**
|
||||
* flush current buffer, clear content
|
||||
*/
|
||||
inline void flush() {
|
||||
if (_remain < _capacity) {
|
||||
flushd();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Memory Key-Value buffer pair with direct address content, so can be
|
||||
* easily copied or dumped to file
|
||||
*/
|
||||
struct KVBuffer {
|
||||
uint32_t keyLength;
|
||||
uint32_t valueLength;
|
||||
char content[1];
|
||||
|
||||
char * getKey() {
|
||||
return content;
|
||||
}
|
||||
|
||||
char * getValue() {
|
||||
return content + keyLength;
|
||||
}
|
||||
|
||||
KVBuffer * next() {
|
||||
return ((KVBuffer*)(content + keyLength + valueLength));
|
||||
}
|
||||
|
||||
std::string str() {
|
||||
return std::string(content, keyLength) + "\t" + std::string(getValue(), valueLength);
|
||||
}
|
||||
|
||||
uint32_t length() {
|
||||
return keyLength + valueLength + SIZE_OF_KEY_LENGTH + SIZE_OF_VALUE_LENGTH;
|
||||
}
|
||||
|
||||
uint32_t lengthConvertEndium() {
|
||||
long value = bswap64(*((long *)this));
|
||||
return (value >> 32) + value + SIZE_OF_KEY_LENGTH + SIZE_OF_VALUE_LENGTH;
|
||||
}
|
||||
|
||||
void fill(const void * key, uint32_t keylen, const void * value, uint32_t vallen) {
|
||||
keyLength = keylen;
|
||||
valueLength = vallen;
|
||||
|
||||
if (keylen > 0) {
|
||||
simple_memcpy(getKey(), key, keylen);
|
||||
}
|
||||
if (vallen > 0) {
|
||||
simple_memcpy(getValue(), value, vallen);
|
||||
}
|
||||
}
|
||||
|
||||
static uint32_t headerLength() {
|
||||
return SIZE_OF_KEY_LENGTH + SIZE_OF_VALUE_LENGTH;
|
||||
}
|
||||
};
|
||||
|
||||
struct KVBufferWithParititionId {
|
||||
uint32_t partitionId;
|
||||
KVBuffer buffer;
|
||||
|
||||
inline static uint32_t minLength() {
|
||||
return SIZE_OF_PARTITION_LENGTH + SIZE_OF_KV_LENGTH;
|
||||
}
|
||||
|
||||
int length() {
|
||||
return 4 + buffer.length();
|
||||
}
|
||||
|
||||
int lengthConvertEndium() {
|
||||
return 4 + buffer.lengthConvertEndium();
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Native side abstraction of java ByteBuffer
|
||||
*/
|
||||
class ByteBuffer {
|
||||
private:
|
||||
char * _buff;
|
||||
uint32_t _limit;
|
||||
uint32_t _position;
|
||||
uint32_t _capacity;
|
||||
|
||||
public:
|
||||
ByteBuffer()
|
||||
: _buff(NULL), _limit(0), _position(0), _capacity(0) {
|
||||
}
|
||||
|
||||
~ByteBuffer() {
|
||||
}
|
||||
|
||||
void reset(char * buff, uint32_t inputCapacity) {
|
||||
this->_buff = buff;
|
||||
this->_capacity = inputCapacity;
|
||||
this->_position = 0;
|
||||
this->_limit = 0;
|
||||
}
|
||||
|
||||
int capacity() {
|
||||
return this->_capacity;
|
||||
}
|
||||
|
||||
int remain() {
|
||||
return _limit - _position;
|
||||
}
|
||||
|
||||
int limit() {
|
||||
return _limit;
|
||||
}
|
||||
|
||||
int advance(int positionOffset) {
|
||||
_position += positionOffset;
|
||||
return _position;
|
||||
}
|
||||
|
||||
int position() {
|
||||
return this->_position;
|
||||
}
|
||||
|
||||
void position(int newPos) {
|
||||
this->_position = newPos;
|
||||
}
|
||||
|
||||
void rewind(int newPos, int newLimit) {
|
||||
this->_position = newPos;
|
||||
if (newLimit < 0 || newLimit > this->_capacity) {
|
||||
THROW_EXCEPTION(IOException, "length smaller than zero or larger than input buffer capacity");
|
||||
}
|
||||
this->_limit = newLimit;
|
||||
}
|
||||
|
||||
char * current() {
|
||||
return _buff + _position;
|
||||
}
|
||||
|
||||
char * base() {
|
||||
return _buff;
|
||||
}
|
||||
};
|
||||
|
||||
class ByteArray {
|
||||
private:
|
||||
char * _buff;
|
||||
uint32_t _length;
|
||||
uint32_t _capacity;
|
||||
|
||||
public:
|
||||
ByteArray()
|
||||
: _buff(NULL), _length(0), _capacity(0) {
|
||||
}
|
||||
|
||||
~ByteArray() {
|
||||
if (NULL != _buff) {
|
||||
delete[] _buff;
|
||||
_buff = NULL;
|
||||
}
|
||||
_length = 0;
|
||||
_capacity = 0;
|
||||
}
|
||||
|
||||
void resize(uint32_t newSize) {
|
||||
if (newSize <= _capacity) {
|
||||
_length = newSize;
|
||||
} else {
|
||||
if (NULL != _buff) {
|
||||
delete[] _buff;
|
||||
_buff = NULL;
|
||||
}
|
||||
_capacity = 2 * newSize;
|
||||
_buff = new char[_capacity];
|
||||
_length = newSize;
|
||||
}
|
||||
}
|
||||
|
||||
char * buff() {
|
||||
return _buff;
|
||||
}
|
||||
|
||||
uint32_t size() {
|
||||
return _length;
|
||||
}
|
||||
};
|
||||
|
||||
class FixSizeContainer {
|
||||
private:
|
||||
char * _buff;
|
||||
uint32_t _pos;
|
||||
uint32_t _size;
|
||||
|
||||
public:
|
||||
FixSizeContainer()
|
||||
: _buff(NULL), _pos(0), _size(0) {
|
||||
}
|
||||
|
||||
~FixSizeContainer() {
|
||||
}
|
||||
|
||||
void wrap(char * buff, uint32_t size) {
|
||||
_size = size;
|
||||
_buff = buff;
|
||||
_pos = 0;
|
||||
}
|
||||
|
||||
void rewind() {
|
||||
_pos = 0;
|
||||
}
|
||||
|
||||
uint32_t remain() {
|
||||
return _size - _pos;
|
||||
}
|
||||
|
||||
char * current() {
|
||||
return _buff + _pos;
|
||||
}
|
||||
|
||||
char * base() {
|
||||
return _buff;
|
||||
}
|
||||
|
||||
uint32_t size() {
|
||||
return _size;
|
||||
}
|
||||
|
||||
/**
|
||||
* return the length of actually filled data.
|
||||
*/
|
||||
uint32_t fill(const char * source, uint32_t maxSize) {
|
||||
int remain = _size - _pos;
|
||||
if (remain <= 0) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
uint32_t length = (maxSize < remain) ? maxSize : remain;
|
||||
simple_memcpy(_buff + _pos, source, length);
|
||||
_pos += length;
|
||||
return length;
|
||||
}
|
||||
|
||||
uint32_t position() {
|
||||
return _pos;
|
||||
}
|
||||
|
||||
void position(int pos) {
|
||||
_pos = pos;
|
||||
}
|
||||
};
|
||||
|
||||
class ReadWriteBuffer {
|
||||
private:
|
||||
|
||||
static const uint32_t INITIAL_LENGTH = 16;
|
||||
|
||||
uint32_t _readPoint;
|
||||
uint32_t _writePoint;
|
||||
char * _buff;
|
||||
uint32_t _buffLength;
|
||||
bool _newCreatedBuff;
|
||||
|
||||
public:
|
||||
|
||||
ReadWriteBuffer(uint32_t length)
|
||||
: _readPoint(0), _writePoint(0), _buff(NULL), _buffLength(0), _newCreatedBuff(false) {
|
||||
_buffLength = length;
|
||||
if (_buffLength > 0) {
|
||||
_buff = new char[_buffLength];
|
||||
_newCreatedBuff = true;
|
||||
}
|
||||
}
|
||||
|
||||
ReadWriteBuffer()
|
||||
: _readPoint(0), _writePoint(0), _buff(NULL), _buffLength(0), _newCreatedBuff(false) {
|
||||
}
|
||||
|
||||
~ReadWriteBuffer() {
|
||||
if (_newCreatedBuff) {
|
||||
delete[] _buff;
|
||||
_buff = NULL;
|
||||
}
|
||||
}
|
||||
|
||||
void setReadPoint(uint32_t pos) {
|
||||
_readPoint = pos;
|
||||
}
|
||||
|
||||
void setWritePoint(uint32_t pos) {
|
||||
_writePoint = pos;
|
||||
}
|
||||
|
||||
char * getBuff() {
|
||||
return _buff;
|
||||
}
|
||||
|
||||
uint32_t getWritePoint() {
|
||||
return _writePoint;
|
||||
}
|
||||
|
||||
uint32_t getReadPoint() {
|
||||
return _readPoint;
|
||||
}
|
||||
|
||||
void writeInt(uint32_t param) {
|
||||
uint32_t written = param;
|
||||
|
||||
checkWriteSpaceAndResizeIfNecessary(4);
|
||||
*((uint32_t *)(_buff + _writePoint)) = written;
|
||||
_writePoint += 4;
|
||||
}
|
||||
|
||||
void writeLong(uint64_t param) {
|
||||
uint64_t written = param;
|
||||
|
||||
checkWriteSpaceAndResizeIfNecessary(8);
|
||||
*((uint64_t *)(_buff + _writePoint)) = written;
|
||||
_writePoint += 8;
|
||||
}
|
||||
|
||||
void writeString(const char * param, uint32_t length) {
|
||||
writeInt(length);
|
||||
checkWriteSpaceAndResizeIfNecessary(length);
|
||||
|
||||
memcpy(_buff + _writePoint, param, length);
|
||||
_writePoint += length;
|
||||
}
|
||||
|
||||
void writeString(std::string * param) {
|
||||
const char * str = param->c_str();
|
||||
int length = param->size();
|
||||
writeString(str, length);
|
||||
}
|
||||
|
||||
void writePointer(void * param) {
|
||||
uint64_t written = (uint64_t)(param);
|
||||
writeLong(written);
|
||||
}
|
||||
|
||||
uint32_t readInt() {
|
||||
char * readPos = _buff + _readPoint;
|
||||
uint32_t result = *((uint32_t *)(readPos));
|
||||
_readPoint += 4;
|
||||
return result;
|
||||
}
|
||||
|
||||
uint64_t readLong() {
|
||||
char * readPos = _buff + _readPoint;
|
||||
uint64_t result = *((uint64_t *)(readPos));
|
||||
_readPoint += 8;
|
||||
return result;
|
||||
}
|
||||
|
||||
std::string * readString() {
|
||||
uint32_t len = readInt();
|
||||
char * strBegin = _buff + _readPoint;
|
||||
_readPoint += len;
|
||||
return new std::string(strBegin, len);
|
||||
}
|
||||
|
||||
void * readPointer() {
|
||||
uint64_t result = readLong();
|
||||
return (void *)(result);
|
||||
}
|
||||
|
||||
private:
|
||||
void checkWriteSpaceAndResizeIfNecessary(uint32_t toBeWritten) {
|
||||
if (_buffLength == 0) {
|
||||
_newCreatedBuff = true;
|
||||
_buffLength = INITIAL_LENGTH > toBeWritten ? INITIAL_LENGTH : toBeWritten;
|
||||
_buff = new char[_buffLength];
|
||||
}
|
||||
|
||||
if (_buffLength - _writePoint >= toBeWritten) {
|
||||
return;
|
||||
}
|
||||
|
||||
_buffLength = _buffLength + toBeWritten;
|
||||
_newCreatedBuff = true;
|
||||
char * newBuff = new char[_buffLength];
|
||||
memcpy(newBuff, _buff, _writePoint);
|
||||
delete[] _buff;
|
||||
_buff = newBuff;
|
||||
}
|
||||
};
|
||||
|
||||
typedef ReadWriteBuffer ParameterBuffer;
|
||||
typedef ReadWriteBuffer ResultBuffer;
|
||||
|
||||
} // namespace NativeTask
|
||||
|
||||
#endif /* BUFFERS_H_ */
|
|
@ -0,0 +1,70 @@
|
|||
/**
|
||||
* 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 COMBINER_H_
|
||||
#define COMBINER_H_
|
||||
#include "commons.h"
|
||||
#include "lib/IFile.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
class MemoryBufferKVIterator : public KVIterator {
|
||||
public:
|
||||
virtual const char * getBase() = 0;
|
||||
virtual std::vector<uint32_t> * getKVOffsets() = 0;
|
||||
};
|
||||
|
||||
enum CombineContextType {
|
||||
UNKNOWN = 0,
|
||||
CONTINUOUS_MEMORY_BUFFER = 1,
|
||||
};
|
||||
|
||||
class CombineContext {
|
||||
|
||||
private:
|
||||
CombineContextType _type;
|
||||
|
||||
public:
|
||||
CombineContext(CombineContextType type)
|
||||
: _type(type) {
|
||||
}
|
||||
|
||||
public:
|
||||
CombineContextType getType() {
|
||||
return _type;
|
||||
}
|
||||
};
|
||||
|
||||
class CombineInMemory : public CombineContext {
|
||||
CombineInMemory()
|
||||
: CombineContext(CONTINUOUS_MEMORY_BUFFER) {
|
||||
}
|
||||
};
|
||||
|
||||
class ICombineRunner {
|
||||
public:
|
||||
ICombineRunner() {
|
||||
}
|
||||
|
||||
virtual void combine(CombineContext type, KVIterator * kvIterator, IFileWriter * writer) = 0;
|
||||
|
||||
virtual ~ICombineRunner() {
|
||||
}
|
||||
};
|
||||
|
||||
} /* namespace NativeTask */
|
||||
#endif /* COMBINER_H_ */
|
|
@ -0,0 +1,145 @@
|
|||
/**
|
||||
* 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 "lib/commons.h"
|
||||
#include "config.h"
|
||||
#include "lib/Compressions.h"
|
||||
#include "util/SyncUtils.h"
|
||||
#include "codec/GzipCodec.h"
|
||||
#include "codec/SnappyCodec.h"
|
||||
#include "codec/Lz4Codec.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
CompressStream::~CompressStream() {
|
||||
}
|
||||
|
||||
void CompressStream::writeDirect(const void * buff, uint32_t length) {
|
||||
THROW_EXCEPTION(UnsupportException, "writeDirect not support");
|
||||
}
|
||||
|
||||
///////////////////////////////////////////////////////////
|
||||
|
||||
DecompressStream::~DecompressStream() {
|
||||
}
|
||||
|
||||
int32_t DecompressStream::readDirect(void * buff, uint32_t length) {
|
||||
THROW_EXCEPTION(UnsupportException, "readDirect not support");
|
||||
}
|
||||
|
||||
///////////////////////////////////////////////////////////
|
||||
|
||||
const Compressions::Codec Compressions::GzipCodec = Compressions::Codec(
|
||||
"org.apache.hadoop.io.compress.GzipCodec", ".gz");
|
||||
const Compressions::Codec Compressions::SnappyCodec = Compressions::Codec(
|
||||
"org.apache.hadoop.io.compress.SnappyCodec", ".snappy");
|
||||
const Compressions::Codec Compressions::Lz4Codec = Compressions::Codec(
|
||||
"org.apache.hadoop.io.compress.Lz4Codec", ".lz4");
|
||||
|
||||
vector<Compressions::Codec> Compressions::SupportedCodecs = vector<Compressions::Codec>();
|
||||
|
||||
void Compressions::initCodecs() {
|
||||
static Lock lock;
|
||||
ScopeLock<Lock> autolock(lock);
|
||||
if (SupportedCodecs.size() == 0) {
|
||||
SupportedCodecs.push_back(GzipCodec);
|
||||
SupportedCodecs.push_back(SnappyCodec);
|
||||
SupportedCodecs.push_back(Lz4Codec);
|
||||
}
|
||||
}
|
||||
|
||||
bool Compressions::support(const string & codec) {
|
||||
initCodecs();
|
||||
for (size_t i = 0; i < SupportedCodecs.size(); i++) {
|
||||
if (codec == SupportedCodecs[i].name) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
const string Compressions::getExtension(const string & codec) {
|
||||
initCodecs();
|
||||
for (size_t i = 0; i < SupportedCodecs.size(); i++) {
|
||||
if (codec == SupportedCodecs[i].name) {
|
||||
return SupportedCodecs[i].extension;
|
||||
}
|
||||
}
|
||||
return string();
|
||||
}
|
||||
|
||||
const string Compressions::getCodec(const string & extension) {
|
||||
initCodecs();
|
||||
for (size_t i = 0; i < SupportedCodecs.size(); i++) {
|
||||
if (extension == SupportedCodecs[i].extension) {
|
||||
return SupportedCodecs[i].name;
|
||||
}
|
||||
}
|
||||
return string();
|
||||
}
|
||||
|
||||
const string Compressions::getCodecByFile(const string & file) {
|
||||
initCodecs();
|
||||
for (size_t i = 0; i < SupportedCodecs.size(); i++) {
|
||||
const string & extension = SupportedCodecs[i].extension;
|
||||
if ((file.length() > extension.length())
|
||||
&& (file.substr(file.length() - extension.length()) == extension)) {
|
||||
return SupportedCodecs[i].name;
|
||||
}
|
||||
}
|
||||
return string();
|
||||
}
|
||||
|
||||
CompressStream * Compressions::getCompressionStream(const string & codec, OutputStream * stream,
|
||||
uint32_t bufferSizeHint) {
|
||||
if (codec == GzipCodec.name) {
|
||||
return new GzipCompressStream(stream, bufferSizeHint);
|
||||
}
|
||||
if (codec == SnappyCodec.name) {
|
||||
#if defined HADOOP_SNAPPY_LIBRARY
|
||||
return new SnappyCompressStream(stream, bufferSizeHint);
|
||||
#else
|
||||
THROW_EXCEPTION(UnsupportException, "Snappy library is not loaded");
|
||||
#endif
|
||||
}
|
||||
if (codec == Lz4Codec.name) {
|
||||
return new Lz4CompressStream(stream, bufferSizeHint);
|
||||
}
|
||||
return NULL;
|
||||
}
|
||||
|
||||
DecompressStream * Compressions::getDecompressionStream(const string & codec, InputStream * stream,
|
||||
uint32_t bufferSizeHint) {
|
||||
if (codec == GzipCodec.name) {
|
||||
return new GzipDecompressStream(stream, bufferSizeHint);
|
||||
}
|
||||
if (codec == SnappyCodec.name) {
|
||||
#if defined HADOOP_SNAPPY_LIBRARY
|
||||
return new SnappyDecompressStream(stream, bufferSizeHint);
|
||||
#else
|
||||
THROW_EXCEPTION(UnsupportException, "Snappy library is not loaded");
|
||||
#endif
|
||||
}
|
||||
if (codec == Lz4Codec.name) {
|
||||
return new Lz4DecompressStream(stream, bufferSizeHint);
|
||||
}
|
||||
return NULL;
|
||||
}
|
||||
|
||||
} // namespace NativeTask
|
||||
|
|
@ -0,0 +1,108 @@
|
|||
/**
|
||||
* 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 COMPRESSIONS_H_
|
||||
#define COMPRESSIONS_H_
|
||||
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include "lib/Streams.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
using std::vector;
|
||||
using std::string;
|
||||
|
||||
class CompressStream : public FilterOutputStream {
|
||||
public:
|
||||
CompressStream(OutputStream * stream)
|
||||
: FilterOutputStream(stream) {
|
||||
}
|
||||
|
||||
virtual ~CompressStream();
|
||||
|
||||
virtual void writeDirect(const void * buff, uint32_t length);
|
||||
|
||||
virtual void finish() {
|
||||
flush();
|
||||
}
|
||||
|
||||
virtual void resetState() {
|
||||
|
||||
}
|
||||
|
||||
virtual uint64_t compressedBytesWritten() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
class DecompressStream : public FilterInputStream {
|
||||
public:
|
||||
DecompressStream(InputStream * stream)
|
||||
: FilterInputStream(stream) {
|
||||
}
|
||||
|
||||
virtual ~DecompressStream();
|
||||
|
||||
virtual int32_t readDirect(void * buff, uint32_t length);
|
||||
|
||||
virtual uint64_t compressedBytesRead() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
class Compressions {
|
||||
protected:
|
||||
class Codec {
|
||||
public:
|
||||
string name;
|
||||
string extension;
|
||||
|
||||
Codec(const string & name, const string & extension)
|
||||
: name(name), extension(extension) {
|
||||
}
|
||||
};
|
||||
|
||||
static vector<Codec> SupportedCodecs;
|
||||
|
||||
static void initCodecs();
|
||||
|
||||
public:
|
||||
static const Codec GzipCodec;
|
||||
static const Codec SnappyCodec;
|
||||
static const Codec Lz4Codec;
|
||||
|
||||
public:
|
||||
static bool support(const string & codec);
|
||||
|
||||
static const string getExtension(const string & codec);
|
||||
|
||||
static const string getCodec(const string & extension);
|
||||
|
||||
static const string getCodecByFile(const string & file);
|
||||
|
||||
static CompressStream * getCompressionStream(const string & codec, OutputStream * stream,
|
||||
uint32_t bufferSizeHint);
|
||||
|
||||
static DecompressStream * getDecompressionStream(const string & codec, InputStream * stream,
|
||||
uint32_t bufferSizeHint);
|
||||
};
|
||||
|
||||
} // namespace NativeTask
|
||||
|
||||
#endif /* COMPRESSIONS_H_ */
|
|
@ -0,0 +1,26 @@
|
|||
/**
|
||||
* 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 CONSTANTS_H_
|
||||
#define CONSTANTS_H_
|
||||
|
||||
const uint32_t SIZE_OF_PARTITION_LENGTH = sizeof(uint32_t);
|
||||
const uint32_t SIZE_OF_KEY_LENGTH = sizeof(uint32_t);
|
||||
const uint32_t SIZE_OF_VALUE_LENGTH = sizeof(uint32_t);
|
||||
const uint32_t SIZE_OF_KV_LENGTH = SIZE_OF_KEY_LENGTH + SIZE_OF_VALUE_LENGTH;
|
||||
|
||||
#endif //CONSTANTS_H_
|
|
@ -0,0 +1,263 @@
|
|||
/*
|
||||
* 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 <errno.h>
|
||||
#include <fcntl.h>
|
||||
#include <dirent.h>
|
||||
#include <sys/stat.h>
|
||||
#include "lib/commons.h"
|
||||
#include "util/StringUtil.h"
|
||||
#include "lib/jniutils.h"
|
||||
#include "NativeTask.h"
|
||||
#include "lib/TaskCounters.h"
|
||||
#include "lib/NativeObjectFactory.h"
|
||||
#include "lib/Path.h"
|
||||
#include "lib/FileSystem.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
/////////////////////////////////////////////////////////////
|
||||
|
||||
FileInputStream::FileInputStream(const string & path) {
|
||||
_fd = ::open(path.c_str(), O_RDONLY);
|
||||
if (_fd >= 0) {
|
||||
_path = path;
|
||||
} else {
|
||||
_fd = -1;
|
||||
THROW_EXCEPTION_EX(IOException, "Can't open file for read: [%s]", path.c_str());
|
||||
}
|
||||
_bytesRead = NativeObjectFactory::GetCounter(TaskCounters::FILESYSTEM_COUNTER_GROUP,
|
||||
TaskCounters::FILE_BYTES_READ);
|
||||
}
|
||||
|
||||
FileInputStream::~FileInputStream() {
|
||||
close();
|
||||
}
|
||||
|
||||
void FileInputStream::seek(uint64_t position) {
|
||||
::lseek(_fd, position, SEEK_SET);
|
||||
}
|
||||
|
||||
uint64_t FileInputStream::tell() {
|
||||
return ::lseek(_fd, 0, SEEK_CUR);
|
||||
}
|
||||
|
||||
int32_t FileInputStream::read(void * buff, uint32_t length) {
|
||||
int32_t ret = ::read(_fd, buff, length);
|
||||
if (ret > 0) {
|
||||
_bytesRead->increase(ret);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void FileInputStream::close() {
|
||||
if (_fd >= 0) {
|
||||
::close(_fd);
|
||||
_fd = -1;
|
||||
}
|
||||
}
|
||||
|
||||
/////////////////////////////////////////////////////////////
|
||||
|
||||
FileOutputStream::FileOutputStream(const string & path, bool overwite) {
|
||||
int flags = 0;
|
||||
if (overwite) {
|
||||
flags = O_WRONLY | O_CREAT | O_TRUNC;
|
||||
} else {
|
||||
flags = O_WRONLY | O_CREAT | O_EXCL;
|
||||
}
|
||||
mode_t mask = umask(0);
|
||||
umask(mask);
|
||||
_fd = ::open(path.c_str(), flags, (0666 & ~mask));
|
||||
if (_fd >= 0) {
|
||||
_path = path;
|
||||
} else {
|
||||
_fd = -1;
|
||||
THROW_EXCEPTION_EX(IOException, "Can't open file for write: [%s]", path.c_str());
|
||||
}
|
||||
_bytesWrite = NativeObjectFactory::GetCounter(TaskCounters::FILESYSTEM_COUNTER_GROUP,
|
||||
TaskCounters::FILE_BYTES_WRITTEN);
|
||||
}
|
||||
|
||||
FileOutputStream::~FileOutputStream() {
|
||||
close();
|
||||
}
|
||||
|
||||
uint64_t FileOutputStream::tell() {
|
||||
return ::lseek(_fd, 0, SEEK_CUR);
|
||||
}
|
||||
|
||||
void FileOutputStream::write(const void * buff, uint32_t length) {
|
||||
if (::write(_fd, buff, length) < length) {
|
||||
THROW_EXCEPTION(IOException, "::write error");
|
||||
}
|
||||
_bytesWrite->increase(length);
|
||||
}
|
||||
|
||||
void FileOutputStream::flush() {
|
||||
}
|
||||
|
||||
void FileOutputStream::close() {
|
||||
if (_fd >= 0) {
|
||||
::close(_fd);
|
||||
_fd = -1;
|
||||
}
|
||||
}
|
||||
|
||||
/////////////////////////////////////////////////////////////
|
||||
|
||||
class RawFileSystem : public FileSystem {
|
||||
protected:
|
||||
string getRealPath(const string & path) {
|
||||
if (StringUtil::StartsWith(path, "file:")) {
|
||||
return path.substr(5);
|
||||
}
|
||||
return path;
|
||||
}
|
||||
|
||||
public:
|
||||
InputStream * open(const string & path) {
|
||||
return new FileInputStream(getRealPath(path));
|
||||
}
|
||||
|
||||
OutputStream * create(const string & path, bool overwrite) {
|
||||
string np = getRealPath(path);
|
||||
string parent = Path::GetParent(np);
|
||||
if (parent.length() > 0) {
|
||||
if (!exists(parent)) {
|
||||
mkdirs(parent);
|
||||
}
|
||||
}
|
||||
return new FileOutputStream(np, overwrite);
|
||||
}
|
||||
|
||||
uint64_t getLength(const string & path) {
|
||||
struct stat st;
|
||||
if (::stat(getRealPath(path).c_str(), &st) != 0) {
|
||||
char buff[256];
|
||||
strerror_r(errno, buff, 256);
|
||||
THROW_EXCEPTION(IOException,
|
||||
StringUtil::Format("stat path %s failed, %s", path.c_str(), buff));
|
||||
}
|
||||
return st.st_size;
|
||||
}
|
||||
|
||||
bool list(const string & path, vector<FileEntry> & status) {
|
||||
DIR * dp;
|
||||
struct dirent * dirp;
|
||||
if ((dp = opendir(path.c_str())) == NULL) {
|
||||
return false;
|
||||
}
|
||||
|
||||
FileEntry temp;
|
||||
while ((dirp = readdir(dp)) != NULL) {
|
||||
temp.name = dirp->d_name;
|
||||
temp.isDirectory = dirp->d_type & DT_DIR;
|
||||
if (temp.name == "." || temp.name == "..") {
|
||||
continue;
|
||||
}
|
||||
status.push_back(temp);
|
||||
}
|
||||
closedir(dp);
|
||||
return true;
|
||||
}
|
||||
|
||||
void remove(const string & path) {
|
||||
if (!exists(path)) {
|
||||
LOG("[FileSystem] remove file %s not exists, ignore", path.c_str());
|
||||
return;
|
||||
}
|
||||
if (::remove(getRealPath(path).c_str()) != 0) {
|
||||
int err = errno;
|
||||
if (::system(StringUtil::Format("rm -rf %s", path.c_str()).c_str()) == 0) {
|
||||
return;
|
||||
}
|
||||
char buff[256];
|
||||
strerror_r(err, buff, 256);
|
||||
THROW_EXCEPTION(IOException,
|
||||
StringUtil::Format("FileSystem: remove path %s failed, %s", path.c_str(), buff));
|
||||
}
|
||||
}
|
||||
|
||||
bool exists(const string & path) {
|
||||
struct stat st;
|
||||
if (::stat(getRealPath(path).c_str(), &st) != 0) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
int mkdirs(const string & path, mode_t nmode) {
|
||||
string np = getRealPath(path);
|
||||
struct stat sb;
|
||||
|
||||
if (stat(np.c_str(), &sb) == 0) {
|
||||
if (S_ISDIR(sb.st_mode) == 0) {
|
||||
return 1;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
string npathstr = np;
|
||||
char * npath = const_cast<char*>(npathstr.c_str());
|
||||
|
||||
/* Skip leading slashes. */
|
||||
char * p = npath;
|
||||
while (*p == '/')
|
||||
p++;
|
||||
|
||||
while (NULL != (p = strchr(p, '/'))) {
|
||||
*p = '\0';
|
||||
if (stat(npath, &sb) != 0) {
|
||||
if (mkdir(npath, nmode)) {
|
||||
return 1;
|
||||
}
|
||||
} else if (S_ISDIR(sb.st_mode) == 0) {
|
||||
return 1;
|
||||
}
|
||||
*p++ = '/'; /* restore slash */
|
||||
while (*p == '/')
|
||||
p++;
|
||||
}
|
||||
|
||||
/* Create the final directory component. */
|
||||
if (stat(npath, &sb) && mkdir(npath, nmode)) {
|
||||
return 1;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
void mkdirs(const string & path) {
|
||||
int ret = mkdirs(path, 0755);
|
||||
if (ret != 0) {
|
||||
THROW_EXCEPTION_EX(IOException, "mkdirs [%s] failed", path.c_str());
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
///////////////////////////////////////////////////////////
|
||||
|
||||
extern RawFileSystem RawFileSystemInstance;
|
||||
|
||||
RawFileSystem RawFileSystemInstance = RawFileSystem();
|
||||
|
||||
FileSystem & FileSystem::getLocal() {
|
||||
return RawFileSystemInstance;
|
||||
}
|
||||
|
||||
} // namespace NativeTask
|
|
@ -0,0 +1,123 @@
|
|||
/**
|
||||
* 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 FILESYSTEM_H_
|
||||
#define FILESYSTEM_H_
|
||||
|
||||
#include <string>
|
||||
#include "NativeTask.h"
|
||||
#include "lib/Streams.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
class FileSystem;
|
||||
|
||||
/**
|
||||
* Local raw filesystem file input stream
|
||||
* with blocking semantics
|
||||
*/
|
||||
class FileInputStream : public InputStream {
|
||||
private:
|
||||
string _path;
|
||||
int _fd;
|
||||
Counter * _bytesRead;
|
||||
public:
|
||||
FileInputStream(const string & path);
|
||||
virtual ~FileInputStream();
|
||||
|
||||
virtual void seek(uint64_t position);
|
||||
|
||||
virtual uint64_t tell();
|
||||
|
||||
virtual int32_t read(void * buff, uint32_t length);
|
||||
|
||||
virtual void close();
|
||||
};
|
||||
|
||||
/**
|
||||
* Local raw filesystem file output stream
|
||||
* with blocking semantics
|
||||
*/
|
||||
class FileOutputStream : public OutputStream {
|
||||
private:
|
||||
string _path;
|
||||
int _fd;
|
||||
Counter * _bytesWrite;
|
||||
public:
|
||||
FileOutputStream(const string & path, bool overwite = true);
|
||||
virtual ~FileOutputStream();
|
||||
|
||||
virtual uint64_t tell();
|
||||
|
||||
virtual void write(const void * buff, uint32_t length);
|
||||
|
||||
virtual void flush();
|
||||
|
||||
virtual void close();
|
||||
};
|
||||
|
||||
|
||||
class FileEntry {
|
||||
public:
|
||||
string name;
|
||||
bool isDirectory;
|
||||
};
|
||||
|
||||
/**
|
||||
* FileSystem interface
|
||||
*/
|
||||
class FileSystem {
|
||||
protected:
|
||||
FileSystem() {
|
||||
}
|
||||
public:
|
||||
virtual ~FileSystem() {
|
||||
}
|
||||
|
||||
virtual InputStream * open(const string & path) {
|
||||
return NULL;
|
||||
}
|
||||
|
||||
virtual OutputStream * create(const string & path, bool overwrite = true) {
|
||||
return NULL;
|
||||
}
|
||||
|
||||
virtual uint64_t getLength(const string & path) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
virtual bool list(const string & path, vector<FileEntry> & status) {
|
||||
return false;
|
||||
}
|
||||
|
||||
virtual void remove(const string & path) {
|
||||
}
|
||||
|
||||
virtual bool exists(const string & path) {
|
||||
return false;
|
||||
}
|
||||
|
||||
virtual void mkdirs(const string & path) {
|
||||
}
|
||||
|
||||
static FileSystem & getLocal();
|
||||
};
|
||||
|
||||
} // namespace NativeTask
|
||||
|
||||
#endif /* FILESYSTEM_H_ */
|
|
@ -0,0 +1,230 @@
|
|||
/**
|
||||
* 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 "lib/commons.h"
|
||||
#include "util/StringUtil.h"
|
||||
#include "lib/IFile.h"
|
||||
#include "lib/Compressions.h"
|
||||
#include "lib/FileSystem.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
///////////////////////////////////////////////////////////
|
||||
|
||||
IFileReader::IFileReader(InputStream * stream, SingleSpillInfo * spill, bool 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), _deleteSourceStream(deleteInputStream) {
|
||||
_source = new ChecksumInputStream(_stream, _checksumType);
|
||||
_source->setLimit(0);
|
||||
_reader.init(128 * 1024, _source, _codec);
|
||||
}
|
||||
|
||||
IFileReader::~IFileReader() {
|
||||
|
||||
delete _source;
|
||||
_source = NULL;
|
||||
|
||||
if (_deleteSourceStream) {
|
||||
delete _stream;
|
||||
_stream = NULL;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 0 if success
|
||||
* 1 if end
|
||||
*/
|
||||
bool IFileReader::nextPartition() {
|
||||
if (0 != _source->getLimit()) {
|
||||
THROW_EXCEPTION(IOException, "bad ifile segment length");
|
||||
}
|
||||
if (_segmentIndex >= 0) {
|
||||
// verify checksum
|
||||
uint32_t chsum = 0;
|
||||
if (4 != _stream->readFully(&chsum, 4)) {
|
||||
THROW_EXCEPTION(IOException, "read ifile checksum failed");
|
||||
}
|
||||
uint32_t actual = bswap(chsum);
|
||||
uint32_t expect = _source->getChecksum();
|
||||
if (actual != expect) {
|
||||
THROW_EXCEPTION_EX(IOException, "read ifile checksum not match, actual %x expect %x", actual,
|
||||
expect);
|
||||
}
|
||||
}
|
||||
_segmentIndex++;
|
||||
if (_segmentIndex < (int)(_spillInfo->length)) {
|
||||
int64_t end_pos = (int64_t)_spillInfo->segments[_segmentIndex].realEndOffset;
|
||||
if (_segmentIndex > 0) {
|
||||
end_pos -= (int64_t)_spillInfo->segments[_segmentIndex - 1].realEndOffset;
|
||||
}
|
||||
if (end_pos < 0) {
|
||||
THROW_EXCEPTION(IOException, "bad ifile format");
|
||||
}
|
||||
// exclude checksum
|
||||
_source->setLimit(end_pos - 4);
|
||||
_source->resetChecksum();
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
///////////////////////////////////////////////////////////
|
||||
|
||||
IFileWriter * IFileWriter::create(const std::string & filepath, const MapOutputSpec & spec,
|
||||
Counter * spilledRecords) {
|
||||
OutputStream * fout = FileSystem::getLocal().create(filepath, true);
|
||||
IFileWriter * writer = new IFileWriter(fout, spec.checksumType, spec.keyType, spec.valueType,
|
||||
spec.codec, spilledRecords, true);
|
||||
return writer;
|
||||
}
|
||||
|
||||
IFileWriter::IFileWriter(OutputStream * stream, ChecksumType checksumType, KeyValueType ktype,
|
||||
KeyValueType vtype, const string & codec, Counter * counter, bool deleteTargetStream)
|
||||
: _stream(stream), _dest(NULL), _checksumType(checksumType), _kType(ktype), _vType(vtype),
|
||||
_codec(codec), _recordCounter(counter), _recordCount(0), _deleteTargetStream(deleteTargetStream) {
|
||||
_dest = new ChecksumOutputStream(_stream, _checksumType);
|
||||
_appendBuffer.init(128 * 1024, _dest, _codec);
|
||||
}
|
||||
|
||||
IFileWriter::~IFileWriter() {
|
||||
delete _dest;
|
||||
_dest = NULL;
|
||||
|
||||
if (_deleteTargetStream) {
|
||||
delete _stream;
|
||||
_stream = NULL;
|
||||
}
|
||||
}
|
||||
|
||||
void IFileWriter::startPartition() {
|
||||
_spillFileSegments.push_back(IFileSegment());
|
||||
_dest->resetChecksum();
|
||||
}
|
||||
|
||||
void IFileWriter::endPartition() {
|
||||
char EOFMarker[2] = {-1, -1};
|
||||
_appendBuffer.write(EOFMarker, 2);
|
||||
_appendBuffer.flush();
|
||||
|
||||
CompressStream * compressionStream = _appendBuffer.getCompressionStream();
|
||||
if (NULL != compressionStream) {
|
||||
compressionStream->finish();
|
||||
compressionStream->resetState();
|
||||
}
|
||||
|
||||
uint32_t chsum = _dest->getChecksum();
|
||||
chsum = bswap(chsum);
|
||||
_stream->write(&chsum, sizeof(chsum));
|
||||
_stream->flush();
|
||||
IFileSegment * info = &(_spillFileSegments[_spillFileSegments.size() - 1]);
|
||||
info->uncompressedEndOffset = _appendBuffer.getCounter();
|
||||
info->realEndOffset = _stream->tell();
|
||||
}
|
||||
|
||||
void IFileWriter::write(const char * key, uint32_t keyLen, const char * value, uint32_t valueLen) {
|
||||
// append KeyLength ValueLength KeyBytesLength
|
||||
uint32_t keyBuffLen = keyLen;
|
||||
uint32_t valBuffLen = valueLen;
|
||||
switch (_kType) {
|
||||
case TextType:
|
||||
keyBuffLen += WritableUtils::GetVLongSize(keyLen);
|
||||
break;
|
||||
case BytesType:
|
||||
keyBuffLen += 4;
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
switch (_vType) {
|
||||
case TextType:
|
||||
valBuffLen += WritableUtils::GetVLongSize(valueLen);
|
||||
break;
|
||||
case BytesType:
|
||||
valBuffLen += 4;
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
_appendBuffer.write_vuint2(keyBuffLen, valBuffLen);
|
||||
|
||||
switch (_kType) {
|
||||
case TextType:
|
||||
_appendBuffer.write_vuint(keyLen);
|
||||
break;
|
||||
case BytesType:
|
||||
_appendBuffer.write_uint32_be(keyLen);
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
if (keyLen > 0) {
|
||||
_appendBuffer.write(key, keyLen);
|
||||
}
|
||||
|
||||
if (NULL != _recordCounter) {
|
||||
_recordCounter->increase();
|
||||
}
|
||||
_recordCount++;
|
||||
|
||||
switch (_vType) {
|
||||
case TextType:
|
||||
_appendBuffer.write_vuint(valueLen);
|
||||
break;
|
||||
case BytesType:
|
||||
_appendBuffer.write_uint32_be(valueLen);
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
if (valueLen > 0) {
|
||||
_appendBuffer.write(value, valueLen);
|
||||
}
|
||||
}
|
||||
|
||||
IFileSegment * IFileWriter::toArray(std::vector<IFileSegment> *segments) {
|
||||
IFileSegment * segs = new IFileSegment[segments->size()];
|
||||
for (size_t i = 0; i < segments->size(); i++) {
|
||||
segs[i] = segments->at(i);
|
||||
}
|
||||
return segs;
|
||||
}
|
||||
|
||||
SingleSpillInfo * IFileWriter::getSpillInfo() {
|
||||
const uint32_t size = _spillFileSegments.size();
|
||||
return new SingleSpillInfo(toArray(&_spillFileSegments), size, "", _checksumType, _kType, _vType,
|
||||
_codec);
|
||||
}
|
||||
|
||||
void IFileWriter::getStatistics(uint64_t & offset, uint64_t & realOffset, uint64_t & recordCount) {
|
||||
if (_spillFileSegments.size() > 0) {
|
||||
offset = _spillFileSegments[_spillFileSegments.size() - 1].uncompressedEndOffset;
|
||||
realOffset = _spillFileSegments[_spillFileSegments.size() - 1].realEndOffset;
|
||||
} else {
|
||||
offset = 0;
|
||||
realOffset = 0;
|
||||
}
|
||||
recordCount = _recordCount;
|
||||
}
|
||||
|
||||
} // namespace NativeTask
|
||||
|
|
@ -0,0 +1,166 @@
|
|||
/**
|
||||
* 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 IFILE_H_
|
||||
#define IFILE_H_
|
||||
|
||||
#include "util/Checksum.h"
|
||||
#include "lib/Buffers.h"
|
||||
#include "util/WritableUtils.h"
|
||||
#include "lib/SpillInfo.h"
|
||||
#include "lib/MapOutputSpec.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
/**
|
||||
* IFileReader
|
||||
*/
|
||||
class IFileReader {
|
||||
private:
|
||||
InputStream * _stream;
|
||||
ChecksumInputStream * _source;
|
||||
ReadBuffer _reader;
|
||||
ChecksumType _checksumType;
|
||||
KeyValueType _kType;
|
||||
KeyValueType _vType;
|
||||
string _codec;
|
||||
int32_t _segmentIndex;
|
||||
SingleSpillInfo * _spillInfo;
|
||||
const char * _valuePos;
|
||||
uint32_t _valueLen;
|
||||
bool _deleteSourceStream;
|
||||
|
||||
public:
|
||||
IFileReader(InputStream * stream, SingleSpillInfo * spill, bool deleteSourceStream = false);
|
||||
|
||||
virtual ~IFileReader();
|
||||
|
||||
/**
|
||||
* @return 0 if have next partition, none 0 if no more partition
|
||||
*/
|
||||
bool nextPartition();
|
||||
|
||||
/**
|
||||
* get next key
|
||||
* NULL if no more, then next_partition() need to be called
|
||||
* NOTICE: before value() is called, the return pointer value is
|
||||
* guaranteed to be valid
|
||||
*/
|
||||
const char * nextKey(uint32_t & keyLen) {
|
||||
int64_t t1 = _reader.readVLong();
|
||||
int64_t t2 = _reader.readVLong();
|
||||
if (t1 == -1) {
|
||||
return NULL;
|
||||
}
|
||||
const char * kvbuff = _reader.get((uint32_t)(t1 + t2));
|
||||
uint32_t len;
|
||||
switch (_kType) {
|
||||
case TextType:
|
||||
keyLen = WritableUtils::ReadVInt(kvbuff, len);
|
||||
break;
|
||||
case BytesType:
|
||||
keyLen = bswap(*(uint32_t*)kvbuff);
|
||||
len = 4;
|
||||
break;
|
||||
default:
|
||||
keyLen = t1;
|
||||
len = 0;
|
||||
}
|
||||
const char * kbuff = kvbuff + len;
|
||||
const char * vbuff = kvbuff + (uint32_t)t1;
|
||||
switch (_vType) {
|
||||
case TextType:
|
||||
_valueLen = WritableUtils::ReadVInt(vbuff, len);
|
||||
_valuePos = vbuff + len;
|
||||
break;
|
||||
case BytesType:
|
||||
_valueLen = bswap(*(uint32_t*)vbuff);
|
||||
_valuePos = vbuff + 4;
|
||||
break;
|
||||
default:
|
||||
_valueLen = t2;
|
||||
_valuePos = vbuff;
|
||||
}
|
||||
return kbuff;
|
||||
}
|
||||
|
||||
/**
|
||||
* length of current value part of IFile entry
|
||||
*/
|
||||
uint32_t valueLen() {
|
||||
return _valueLen;
|
||||
}
|
||||
|
||||
/**
|
||||
* get current value
|
||||
*/
|
||||
const char * value(uint32_t & valueLen) {
|
||||
valueLen = _valueLen;
|
||||
return _valuePos;
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* IFile Writer
|
||||
*/
|
||||
class IFileWriter : public Collector {
|
||||
protected:
|
||||
OutputStream * _stream;
|
||||
ChecksumOutputStream * _dest;
|
||||
ChecksumType _checksumType;
|
||||
KeyValueType _kType;
|
||||
KeyValueType _vType;
|
||||
string _codec;
|
||||
AppendBuffer _appendBuffer;
|
||||
vector<IFileSegment> _spillFileSegments;
|
||||
Counter * _recordCounter;
|
||||
uint64_t _recordCount;
|
||||
|
||||
bool _deleteTargetStream;
|
||||
|
||||
private:
|
||||
IFileSegment * toArray(std::vector<IFileSegment> *segments);
|
||||
|
||||
public:
|
||||
static IFileWriter * create(const std::string & filepath, const MapOutputSpec & spec,
|
||||
Counter * spilledRecords);
|
||||
|
||||
IFileWriter(OutputStream * stream, ChecksumType checksumType, KeyValueType ktype,
|
||||
KeyValueType vtype, const string & codec, Counter * recordCounter,
|
||||
bool deleteTargetStream = false);
|
||||
|
||||
virtual ~IFileWriter();
|
||||
|
||||
void startPartition();
|
||||
|
||||
void endPartition();
|
||||
|
||||
virtual void write(const char * key, uint32_t keyLen, const char * value, uint32_t valueLen);
|
||||
|
||||
SingleSpillInfo * getSpillInfo();
|
||||
|
||||
void getStatistics(uint64_t & offset, uint64_t & realOffset, uint64_t & recordCount);
|
||||
|
||||
virtual void collect(const void * key, uint32_t keyLen, const void * value, uint32_t valueLen) {
|
||||
write((const char*)key, keyLen, (const char*)value, valueLen);
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace NativeTask
|
||||
|
||||
#endif /* IFILE_H_ */
|
|
@ -0,0 +1,91 @@
|
|||
/**
|
||||
* 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 "lib/Iterator.h"
|
||||
#include "lib/commons.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
KeyGroupIteratorImpl::KeyGroupIteratorImpl(KVIterator * iterator)
|
||||
: _keyGroupIterState(NEW_KEY), _iterator(iterator), _first(true) {
|
||||
}
|
||||
|
||||
bool KeyGroupIteratorImpl::nextKey() {
|
||||
if (_keyGroupIterState == NO_MORE) {
|
||||
return false;
|
||||
}
|
||||
|
||||
uint32_t temp;
|
||||
while (_keyGroupIterState == SAME_KEY || _keyGroupIterState == NEW_KEY_VALUE) {
|
||||
nextValue(temp);
|
||||
}
|
||||
if (_keyGroupIterState == NEW_KEY) {
|
||||
if (_first == true) {
|
||||
_first = false;
|
||||
if (!next()) {
|
||||
_keyGroupIterState = NO_MORE;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
_keyGroupIterState = NEW_KEY_VALUE;
|
||||
_currentGroupKey.assign(_key.data(), _key.length());
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
const char * KeyGroupIteratorImpl::getKey(uint32_t & len) {
|
||||
len = (uint32_t)_key.length();
|
||||
return _key.data();
|
||||
}
|
||||
|
||||
const char * KeyGroupIteratorImpl::nextValue(uint32_t & len) {
|
||||
switch (_keyGroupIterState) {
|
||||
case NEW_KEY: {
|
||||
return NULL;
|
||||
}
|
||||
case SAME_KEY: {
|
||||
if (next()) {
|
||||
if (_key.length() == _currentGroupKey.length()) {
|
||||
if (fmemeq(_key.data(), _currentGroupKey.c_str(), _key.length())) {
|
||||
len = _value.length();
|
||||
return _value.data();
|
||||
}
|
||||
}
|
||||
_keyGroupIterState = NEW_KEY;
|
||||
return NULL;
|
||||
}
|
||||
_keyGroupIterState = NO_MORE;
|
||||
return NULL;
|
||||
}
|
||||
case NEW_KEY_VALUE: {
|
||||
_keyGroupIterState = SAME_KEY;
|
||||
len = _value.length();
|
||||
return _value.data();
|
||||
}
|
||||
case NO_MORE:
|
||||
return NULL;
|
||||
}
|
||||
return NULL;
|
||||
}
|
||||
|
||||
bool KeyGroupIteratorImpl::next() {
|
||||
bool result = _iterator->next(_key, _value);
|
||||
return result;
|
||||
}
|
||||
|
||||
} // namespace NativeTask
|
|
@ -0,0 +1,47 @@
|
|||
/**
|
||||
* 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 ITERATOR_H_
|
||||
#define ITERATOR_H_
|
||||
|
||||
#include "NativeTask.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
class KeyGroupIteratorImpl : public KeyGroupIterator {
|
||||
protected:
|
||||
// for KeyGroupIterator
|
||||
KeyGroupIterState _keyGroupIterState;
|
||||
KVIterator * _iterator;
|
||||
string _currentGroupKey;
|
||||
Buffer _key;
|
||||
Buffer _value;
|
||||
bool _first;
|
||||
|
||||
public:
|
||||
KeyGroupIteratorImpl(KVIterator * iterator);
|
||||
bool nextKey();
|
||||
const char * getKey(uint32_t & len);
|
||||
const char * nextValue(uint32_t & len);
|
||||
|
||||
protected:
|
||||
bool next();
|
||||
};
|
||||
|
||||
} //namespace NativeTask
|
||||
#endif
|
|
@ -0,0 +1,30 @@
|
|||
/**
|
||||
* 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 "lib/Log.h"
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
#ifdef PRINT_LOG
|
||||
|
||||
FILE * LOG_DEVICE = stderr;
|
||||
|
||||
#endif
|
||||
|
||||
} //namespace NativeTask
|
||||
|
|
@ -0,0 +1,48 @@
|
|||
/**
|
||||
* 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 LOG_H_
|
||||
#define LOG_H_
|
||||
|
||||
#include <stdio.h>
|
||||
#include <time.h>
|
||||
|
||||
namespace NativeTask {
|
||||
|
||||
#define PRINT_LOG
|
||||
|
||||
#ifdef PRINT_LOG
|
||||
|
||||
extern FILE * LOG_DEVICE;
|
||||
#define LOG(_fmt_, args...) if (LOG_DEVICE) { \
|
||||
time_t log_timer; struct tm log_tm; \
|
||||
time(&log_timer); localtime_r(&log_timer, &log_tm); \
|
||||
fprintf(LOG_DEVICE, "%02d/%02d/%02d %02d:%02d:%02d INFO "_fmt_"\n", \
|
||||
log_tm.tm_year%100, log_tm.tm_mon+1, log_tm.tm_mday, \
|
||||
log_tm.tm_hour, log_tm.tm_min, log_tm.tm_sec, \
|
||||
##args);}
|
||||
|
||||
#else
|
||||
|
||||
#define LOG(_fmt_, args...)
|
||||
|
||||
#endif
|
||||
|
||||
} // namespace NativeTask
|
||||
|
||||
#endif /* LOG_H_ */
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue