Import initial code for MAPREDUCE-2841 (native output collector)

Contributed by:
  Binglin Chang
  Yang Dong
  Sean Zhong
  Manu Zhang
  Zhongliang Zhu
  Vincent Wang
  Yan Dong
  Cheng Lian
  Xusen Yin
  Fangqin Dai
  Jiang Weihua
  Gansha Wu
  Avik Dey


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/MR-2841@1611413 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2014-07-17 17:44:55 +00:00
parent cc57df33b0
commit b2551c06a0
249 changed files with 56959 additions and 4 deletions

View File

@ -226,6 +226,10 @@ public class Text extends BinaryComparable
this.length = len;
}
public void setLength(int len) {
this.length = len;
}
/**
* Append a range of bytes to the end of the given text
* @param utf8 the data to copy from
@ -260,7 +264,7 @@ public class Text extends BinaryComparable
* @param len the number of bytes we need
* @param keepData should the old data be kept
*/
private void setCapacity(int len, boolean keepData) {
public void setCapacity(int len, boolean keepData) {
if (bytes == null || bytes.length < len) {
if (bytes != null && keepData) {
bytes = Arrays.copyOf(bytes, Math.max(len,length << 1));

View File

@ -122,6 +122,7 @@
run cp -r $ROOT/hadoop-hdfs-project/hadoop-hdfs-nfs/target/hadoop-hdfs-nfs-${project.version}/* .
run cp -r $ROOT/hadoop-yarn-project/target/hadoop-yarn-project-${project.version}/* .
run cp -r $ROOT/hadoop-mapreduce-project/target/hadoop-mapreduce-${project.version}/* .
run cp -r $ROOT/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/target/hadoop-mapreduce-client-nativetask-${project.version}/* .
run cp -r $ROOT/hadoop-tools/hadoop-tools-dist/target/hadoop-tools-dist-${project.version}/* .
echo
echo "Hadoop dist layout available at: ${project.build.directory}/hadoop-${project.version}"

View File

@ -385,11 +385,19 @@ public class MapTask extends Task {
ReflectionUtils.newInstance(
job.getClass(JobContext.MAP_OUTPUT_COLLECTOR_CLASS_ATTR,
MapOutputBuffer.class, MapOutputCollector.class), job);
LOG.info("Map output collector class = " + collector.getClass().getName());
MapOutputCollector.Context context =
new MapOutputCollector.Context(this, job, reporter);
collector.init(context);
return collector;
try {
collector.init(context);
} catch (Exception e) {
LOG.warn("Nativetask falling back to Java MapOutputCollector", e);
collector = new MapOutputBuffer();
collector.init(context);
} finally {
LOG.info("Map output collector class = " + collector.getClass().getName());
return collector;
}
}
@SuppressWarnings("unchecked")

View File

@ -0,0 +1,288 @@
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright [yyyy] [name of copyright owner]
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.
Licenses for third party projects used by this project:
CityHash src/main/native/cityhash
---------------------------------------------------------------------
// Copyright (c) 2011 Google, Inc.
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to deal
// in the Software without restriction, including without limitation the rights
// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
// copies of the Software, and to permit persons to whom the Software is
// furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
// THE SOFTWARE.
GoogleTest src/main/native/gtest
---------------------------------------------------------------------
Copyright 2008, Google Inc.
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.
LZ4 src/main/native/lz4
---------------------------------------------------------------------
LZ4 - Fast LZ compression algorithm
Copyright (C) 2011, Yann Collet.
BSD License
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.
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.

View File

@ -0,0 +1,186 @@
<?xml version="1.0"?>
<!--
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>
</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>**/TestTaskContext.java</include>
<include>**/buffer/Test*.java</include>
<include>**/handlers/Test*.java</include>
<include>**/serde/Test*.java</include>
<include>**/combinertest/*Test.java</include>
<include>**/compresstest/*Test.java</include>
<include>**/nonsorttest/*Test.java</include>
<include>**/kvtest/*Test.java</include>
</includes>
<additionalClasspathElements>
<additionalClasspathElement>${basedir}/../hadoop-mapreduce-client-common/target/classes</additionalClasspathElement>
</additionalClasspathElements>
</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 todir="${project.build.directory}/native/test/testData" overwrite="true">
<fileset dir="${basedir}/src/main/native/testData" />
</copy>
<copy file="${basedir}/src/main/native/test.sh" todir="${project.build.directory}/native/test" />
<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>
</executions>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>

View File

@ -0,0 +1 @@
This project is depend on hadoop and hadoop-nativetask, so in order to complete the build, you have to specify the hadoop dir first.

View File

@ -0,0 +1,131 @@
<?xml version="1.0"?>
<!-- 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">
<modelVersion>4.0.0</modelVersion>
<groupId>org.apache.hadoop</groupId>
<artifactId>nativetask-sdk</artifactId>
<version>2.2.0</version>
<name>nativetask-sdk</name>
<properties>
</properties>
<dependencies>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-mapreduce-client-nativetask</artifactId>
<version>2.2.0</version>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
</configuration>
</plugin>
</plugins>
</build>
<profiles>
<profile>
<id>native</id>
<activation>
<activeByDefault>false</activeByDefault>
</activation>
<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.apache.maven.plugins</groupId>
<artifactId>maven-antrun-plugin</artifactId>
<executions>
<execution>
<id>make</id>
<phase>compile</phase>
<goals>
<goal>run</goal>
</goals>
<configuration>
<target>
<exec executable="sh" dir="${basedir}/src/main/native"
failonerror="true">
<arg value="${basedir}/src/main/native/build.sh" />
<arg value="all" />
</exec>
<copy toDir="${project.build.directory}/native/target/usr/local/lib" overwrite="true">
<fileset dir="${basedir}/src/main/native/.libs">
<exclude name="**/*.o"/>
<exclude name="nttest"/>
</fileset>
</copy>
<copy todir="${project.build.directory}/${project.artifactId}-${project.version}/lib/native/sdk/" overwrite="true">
<fileset dir="${basedir}/src/main/native/.libs">
<exclude name="**/*.o"/>
<exclude name="nttest"/>
</fileset>
<fileset file="${basedir}/src/main/native/src/NativeTask.h" />
</copy>
</target>
</configuration>
</execution>
<execution>
<id>make-clean</id>
<phase>clean</phase>
<goals>
<goal>run</goal>
</goals>
<configuration>
<target>
<exec executable="sh" dir="${basedir}/src/main/native"
failonerror="true">
<arg value="${basedir}/src/main/native/build.sh" />
<arg value="clean" />
</exec>
</target>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>

View File

@ -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.nativetask.platform.custom;
import java.io.IOException;
import org.apache.hadoop.mapred.nativetask.Platform;
import org.apache.hadoop.nativetask.serde.custom.CustomWritable;
import org.apache.hadoop.nativetask.serde.custom.CustomWritableSerializer;
public class CustomPlatform extends Platform{
@Override
public void init() throws IOException {
registerKey(CustomWritable.class.getName(),
CustomWritableSerializer.class);
}
@Override
public String name() {
return "CustomPlatform";
}
}

View File

@ -0,0 +1,75 @@
/**
* 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.nativetask.serde.custom;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.io.WritableComparable;
public class CustomWritable implements WritableComparable<CustomWritable>{
private int Id_a;
private long Id_b;
public CustomWritable() {
this.Id_a = 0;
this.Id_b = 0;
}
public CustomWritable(int a, long b){
this.Id_a = a;
this.Id_b = b;
}
@Override
public void readFields(DataInput in) throws IOException {
Id_a = in.readInt();
Id_b = in.readLong();
}
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(Id_a);
out.writeLong(Id_b);
}
@Override
public int compareTo(CustomWritable that) {
if(Id_a > that.Id_a){
return 1;
}
if(Id_a < that.Id_a){
return -1;
}
if(Id_b > that.Id_b){
return 1;
}
if(Id_b < that.Id_b){
return -1;
}
return 0;
}
@Override
public String toString() {
return Id_a + "\t" + Id_b;
}
}

View File

@ -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.nativetask.serde.custom;
import java.io.IOException;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.nativetask.INativeComparable;
import org.apache.hadoop.mapred.nativetask.serde.DefaultSerializer;
public class CustomWritableSerializer extends DefaultSerializer implements
INativeComparable{
@Override
public int getLength(Writable w) throws IOException {
return 12;
}
}

View File

@ -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 <iostream>
#include "NativeTask.h"
using namespace NativeTask;
namespace Custom {
using namespace std;
inline uint32_t bswap(uint32_t val) {
__asm__("bswap %0" : "=r" (val) : "0" (val));
return val;
}
int32_t ReadInt(const char * src) {
return (int32_t) bswap(*(uint32_t*) src);
}
inline uint64_t bswap64(uint64_t val) {
#ifdef __X64
__asm__("bswapq %0" : "=r" (val) : "0" (val));
#else
uint64_t lower = val & 0xffffffffU;
uint32_t higher = (val >> 32) & 0xffffffffU;
lower = bswap(lower);
higher = bswap(higher);
return (lower << 32) + higher;
#endif
return val;
}
int64_t ReadLong(const char * src) {
return (int64_t) bswap64(*(uint64_t*) src);
}
int CustomComparator(const char * src, uint32_t srcLength, const char * dest,
uint32_t destLength) {
int32_t src_IDa = ReadInt(src);
int64_t src_IDb = ReadLong(src+4);
int32_t dest_IDa = ReadInt(dest);
int64_t dest_IDb = ReadLong(dest+4);
if(src_IDa > dest_IDa){
return 1;
}
if(src_IDa < dest_IDa){
return -1;
}
if(src_IDb > dest_IDb){
return 1;
}
if(src_IDb < dest_IDb){
return -1;
}
return 0;
};
DEFINE_NATIVE_LIBRARY(Custom) {
REGISTER_FUNCTION(CustomComparator,Custom);
}
}

View File

@ -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.
#
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} -Wall")
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_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)
include_directories(
${GENERATED_JAVAH}
${D}
${D}/src
${D}/src/util
${D}/src/lib
${D}/test
${CMAKE_CURRENT_SOURCE_DIR}
#${CMAKE_CURRENT_SOURCE_DIR}/src
#${CMAKE_BINARY_DIR}
${JNI_INCLUDE_DIRS}
${SNAPPY_INCLUDE_DIR}
)
#SET(CMAKE_SOURCE_DIR "/cygdrive/c/Users/tianlunz/repo/hadoop-2.2.0-src/hadoop-common-project/hadoop-common/src")
CONFIGURE_FILE(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
SET(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
add_dual_library(nativetask
${D}/lz4/lz4.c
${D}/cityhash/city.cc
${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/Combiner.cc
${D}/src/lib/TaskCounters.cc
${D}/src/util/Checksum.cc
${D}/src/util/Hash.cc
${D}/src/util/Random.cc
${D}/src/util/StringUtil.cc
${D}/src/util/SyncUtils.cc
${D}/src/util/Timer.cc
${D}/src/util/WritableUtils.cc
)
target_link_libraries(nativetask
#${LIB_DL}
dl
rt
pthread
z
${SNAPPY_LIBRARY}
${JAVA_JVM_LIBRARY}
)
add_executable(nttest
${D}/gtest/gtest-all.cc
${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/lib/TestTrackingCollector.cc
${D}/test/util/TestChecksum.cc
${D}/test/util/TestHash.cc
${D}/test/util/TestStringUtil.cc
${D}/test/util/TestSyncUtils.cc
${D}/test/util/TestWritableUtils.cc
${D}/test/TestCommand.cc
${D}/test/TestConfig.cc
${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)
set(CMAKE_EXE_LINKER_FLAGS "-L${_JAVA_HOME}/jre/lib/amd64/server -ljvm")
target_link_libraries(nttest
nativetask_static
dl
rt
pthread
z
${SNAPPY_LIBRARY}
# ${JAVA_JVM_LIBRARY}
)
#if (NEED_LINK_DL)
# set(LIB_DL dl)
#endif (NEED_LINK_DL)
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)

View File

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

View File

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

View File

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

View File

@ -0,0 +1,38 @@
/**
* 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.mapred.nativetask.util.ReadWriteBuffer;
/**
* a CommandDispatcher receives {@link Command} from upstream
* and performs corresponding operations
*/
public interface CommandDispatcher {
/**
*
* @param command
* @param parameter
* @return
* @throws IOException
*/
public ReadWriteBuffer onCall(Command command, ReadWriteBuffer parameter) throws IOException;
}

View File

@ -0,0 +1,59 @@
/**
* 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;
public class Constants {
public static final String MAP_SORT_CLASS = "map.sort.class";
public static final String MAPRED_COMBINER_CLASS = "mapred.combiner.class";
public static final String MAPRED_MAPTASK_DELEGATOR_CLASS = "mapreduce.map.task.delegator.class";
public static final String MAPRED_REDUCETASK_DELEGATOR_CLASS = "mapreduce.reduce.task.delegator.class";
public static final String NATIVE_TASK_ENABLED = "native.task.enabled";
public static final String NATIVE_LOG_DEVICE = "native.log.device";
public static final String NATIVE_HADOOP_VERSION = "native.hadoop.version";
public static final String NATIVE_MAPPER_CLASS = "native.mapper.class";
public static final String NATIVE_REDUCER_CLASS = "native.reducer.class";
public static final String NATIVE_PARTITIONER_CLASS = "native.partitioner.class";
public static final String NATIVE_COMBINER_CLASS = "native.combiner.class";
public static final String NATIVE_INPUT_SPLIT = "native.input.split";
public static final String NATIVE_RECORDREADER_CLASS = "native.recordreader.class";
public static final String NATIVE_RECORDWRITER_CLASS = "native.recordwriter.class";
public static final String NATIVE_OUTPUT_FILE_NAME = "native.output.file.name";
public static final String NATIVE_PROCESSOR_BUFFER_KB = "native.processor.buffer.kb";
public static int NATIVE_PROCESSOR_BUFFER_KB_DEFAULT = 64;
public static int NATIVE_ASYNC_PROCESSOR_BUFFER_KB_DEFAULT = 1024;
public static final String NATIVE_STATUS_UPDATE_INTERVAL = "native.update.interval";
public static int NATIVE_STATUS_UPDATE_INTERVAL_DEFVAL = 3000;
public static final String SERIALIZATION_FRAMEWORK = "SerializationFramework";
public static int SIZEOF_PARTITION_LENGTH = 4;
public static int SIZEOF_KEY_LENGTH = 4;
public static int SIZEOF_VALUE_LENGTH = 4;
public static 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";
}

View File

@ -0,0 +1,38 @@
/**
* 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;
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
}

View File

@ -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;
/**
* a DataReceiver pulls in arriving data, an example
* is {@link org.apache.hadoop.mapred.nativetask.handlers.BufferPuller}
*/
public interface DataReceiver {
/**
* Send a signal to the receiver that the data arrives.
* The data is transferred in another band.
*
* @return
* @throws IOException
*/
public boolean receiveData() throws IOException;
}

View File

@ -0,0 +1,81 @@
/**
* 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.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.*;
import org.apache.log4j.Logger;
public class HadoopPlatform extends Platform {
private static final Logger LOG = Logger.getLogger(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";
}
}

View File

@ -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.
*/
package org.apache.hadoop.mapred.nativetask;
import java.io.IOException;
/**
* interacts with native side to support Java Combiner
*/
public interface ICombineHandler {
/**
* run combiner
* @throws IOException
*/
public void combine() throws IOException;
/**
* @return id of this handler
*/
public long getId();
/**
* close handlers, buffer pullers and pushers
* @throws IOException
*/
public void close() throws IOException;
}

View File

@ -0,0 +1,50 @@
/**
* 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;
/**
*
* Any key type that is comparable at native side must implement this interface
*
* a native comparator function should have the ComparatorPtr type
*
* typedef int (*ComparatorPtr)(const char * src, uint32_t srcLength,
* const char * dest, uint32_t destLength);
*
* 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 {@see HiveKey#write} is serialized as
* int field (containing the length of raw bytes) + raw bytes
* When comparing two HiveKeys, we firstly read the length field and then
* comparing the raw bytes invoking the BytesComparator provided by our library.
* We pass the location and length of raw bytes into BytesComparator
*
* 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);
* }
*/
public interface INativeComparable {
}

View File

@ -0,0 +1,59 @@
/**
* 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.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
*/
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
*
* @param command
* @param parameter
* @return
* @throws IOException
*/
public ReadWriteBuffer call(Command command, ReadWriteBuffer parameter) throws IOException;
/**
* @param handler
*/
void setCommandDispatcher(CommandDispatcher handler);
}

View File

@ -0,0 +1,289 @@
/**
* 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.conf.Configuration;
import org.apache.hadoop.mapred.nativetask.buffer.BufferType;
import org.apache.hadoop.mapred.nativetask.buffer.DirectBufferPool;
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
*/
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;
}
if (null != in && null != in.getByteBuffer() && in.getByteBuffer().isDirect()) {
DirectBufferPool.getInstance().returnBuffer(in.getByteBuffer());
}
}
@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
*
* @param handler
* @param length
*/
private native void nativeProcessInput(long handler, int length);
/**
* Notice native side input is finished
*
* @param handler
*/
private native void nativeFinish(long handler);
/**
* Send control message to native side
*
* @param cmd
* command data
* @return return value
*/
private native byte[] nativeCommand(long handler, int cmd, byte[] parameter);
/**
* Load data from native
*
* @return
*/
private native void nativeLoadData(long handler);
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;
}
}

View File

@ -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.mapred.nativetask.buffer.InputBuffer;
import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer;
/**
* NativeDataSource loads data from upstream
*/
public interface NativeDataSource {
/**
* get input buffer
*
* @return
*/
public InputBuffer getInputBuffer();
/**
* set listener. When data from upstream arrives, the listener will be activated.
*
* @param handler
*/
void setDataReceiver(DataReceiver handler);
/**
* load data from upstream
*
* @return
* @throws IOException
*/
public void loadData() throws IOException;
}

View File

@ -0,0 +1,51 @@
/**
* 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.mapred.nativetask.buffer.OutputBuffer;
/**
* NativeDataTarge sends data to downstream
*/
public interface NativeDataTarget {
/**
* send a signal to indicate that the data has been stored in output buffer
*
* @throws IOException
*/
public void sendData() throws IOException;
/**
* Send a signal that there is no more data
*
* @throws IOException
*/
public void finishSendData() throws IOException;
/**
* get the output buffer.
*
* @return
*/
public OutputBuffer getOutputBuffer();
}

View File

@ -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;
import java.io.File;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
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.server.jobtracker.JTConfig;
import org.apache.hadoop.util.QuickSort;
import org.apache.hadoop.util.RunJar;
/**
* native map output collector wrapped in Java interface
*/
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 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();
}
}
@Override
public void flush() throws IOException, InterruptedException, ClassNotFoundException {
handler.flush();
}
@Override
public void init(Context context) throws IOException, ClassNotFoundException {
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 don't support customized java comparator "
+ job.get(MRJobConfig.KEY_COMPARATOR);
LOG.error(message);
throw new InvalidJobConfException(message);
}
if (job.getBoolean(MRJobConfig.MAP_OUTPUT_COMPRESS, false) == true) {
if (!isCodecSupported(job.get(MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC))) {
String message = "Native output collector don't support compression codec "
+ job.get(MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC) + ", We support Gzip, Lz4, snappy";
LOG.error(message);
throw new InvalidJobConfException(message);
}
}
if (!QuickSort.class.getName().equals(job.get(Constants.MAP_SORT_CLASS))) {
String message = "Native-Task don'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 don'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 don'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) {
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 the 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!");
}
private boolean isCodecSupported(String string) {
if ("org.apache.hadoop.io.compress.SnappyCodec".equals(string)
|| "org.apache.hadoop.io.compress.GzipCodec".equals(string)
|| "org.apache.hadoop.io.compress.Lz4Codec".equals(string)) {
return true;
}
return false;
}
}

View File

@ -0,0 +1,216 @@
/**
* 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.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
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.BytesUtil;
import org.apache.hadoop.mapred.nativetask.util.ConfigUtil;
import org.apache.hadoop.mapred.nativetask.util.SnappyUtil;
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, and 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.
*
*/
public class NativeRuntime {
private static Log LOG = LogFactory.getLog(NativeRuntime.class);
private static boolean nativeLibraryLoaded = false;
private static Configuration conf = new Configuration();
static {
try {
if (false == SnappyUtil.isNativeSnappyLoaded(conf)) {
throw new IOException("Snappy library cannot be loaded");
} else {
LOG.info("Snappy native library is available");
}
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
*
* @param clazz
* @return
*/
public synchronized static long createNativeObject(String clazz) {
assertNativeLibraryLoaded();
final long ret = JNICreateNativeObject(BytesUtil.toBytes(clazz));
if (ret == 0) {
LOG.warn("Can't create NativeObject for class " + clazz + ", probably not exist.");
}
return ret;
}
/**
* Register a customized library
*
* @param clazz
* @return
*/
public synchronized static long registerLibrary(String libraryName, String clazz) {
assertNativeLibraryLoaded();
final long ret = JNIRegisterModule(BytesUtil.toBytes(libraryName), BytesUtil.toBytes(clazz));
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
*
* @param reporter
* @throws IOException
*/
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
********************************************************/
/**
* Config the native runtime with mapreduce job configurations.
*
* @param configs
*/
private native static void JNIConfigure(byte[][] configs);
/**
* create a native object in native space
*
* @param clazz
* @return
*/
private native static long JNICreateNativeObject(byte[] clazz);
/**
* create the default native object for certain type
*
* @param type
* @return
*/
@Deprecated
private native static long JNICreateDefaultNativeObject(byte[] type);
/**
* destroy native object in native space
*
* @param addr
*/
private native static void JNIReleaseNativeObject(long addr);
/**
* get status update from native side Encoding: progress:float status:Text Counter number: int the count of the
* counters Counters: array [group:Text, name:Text, incrCount:Long]
*
* @return
*/
private native static byte[] JNIUpdateStatus();
/**
* Not used.
*/
private native static void JNIRelease();
/**
* Not used.
*/
private native static int JNIRegisterModule(byte[] path, byte[] name);
}

View File

@ -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.
*/
package org.apache.hadoop.mapred.nativetask;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.InvalidJobConfException;
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.
*/
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
*
* @throws IOException
*/
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
* @throws IOException
*/
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 comparator(set with mapreduce.job.output.key.comparator.class)
* but a platform (e.g Pig) could also set that conf and implement native comparators so
* we shouldn't bail out.
*
* @param keyComparator comparator set with mapreduce.job.output.key.comparator.class
* @return
*/
protected abstract boolean define(Class keyComparator);
}

View File

@ -0,0 +1,75 @@
/**
* 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.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;
import org.apache.log4j.Logger;
/**
* 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
*/
public class Platforms {
private static final Logger LOG = Logger.getLogger(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;
}
}

View File

@ -0,0 +1,103 @@
/**
* 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.Counter;
import org.apache.hadoop.mapred.Task.TaskReporter;
/**
* Will periodically check status from native and report to MR framework.
*
*/
public class StatusReportChecker implements Runnable {
private static Log LOG = LogFactory.getLog(StatusReportChecker.class);
public static int INTERVAL = 1000; // milli-seconds
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(Counter.MAP_INPUT_RECORDS);
reporter.getCounter(Counter.MAP_OUTPUT_RECORDS);
reporter.getCounter(Counter.MAP_INPUT_BYTES);
reporter.getCounter(Counter.MAP_OUTPUT_BYTES);
reporter.getCounter(Counter.MAP_OUTPUT_MATERIALIZED_BYTES);
reporter.getCounter(Counter.COMBINE_INPUT_RECORDS);
reporter.getCounter(Counter.COMBINE_OUTPUT_RECORDS);
reporter.getCounter(Counter.REDUCE_INPUT_RECORDS);
reporter.getCounter(Counter.REDUCE_OUTPUT_RECORDS);
reporter.getCounter(Counter.REDUCE_INPUT_GROUPS);
reporter.getCounter(Counter.SPILLED_RECORDS);
reporter.getCounter(Counter.MAP_OUTPUT_BYTES);
reporter.getCounter(Counter.MAP_OUTPUT_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();
}
}
}

View File

@ -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;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.Task.TaskReporter;
import org.apache.hadoop.mapred.TaskAttemptID;
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 getOuputKeyClass() {
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);
}
}

View File

@ -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.
*/
package org.apache.hadoop.mapred.nativetask.buffer;
public enum BufferType {
DIRECT_BUFFER,
HEAP_BUFFER
};

View File

@ -0,0 +1,264 @@
/**
* 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.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.io.PushbackInputStream;
import java.io.UTFDataFormatException;
import java.nio.ByteBuffer;
/**
* read data from a input buffer
*/
public class ByteBufferDataReader extends DataInputStream {
private ByteBuffer byteBuffer;
private char lineCache[];
public ByteBufferDataReader(InputBuffer buffer) {
if (buffer != null) {
this.byteBuffer = buffer.getByteBuffer();
}
}
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();
}
@Override
public String readLine() throws IOException {
InputStream in = this;
char buf[] = lineCache;
if (buf == null) {
buf = lineCache = new char[128];
}
int room = buf.length;
int offset = 0;
int c;
loop: while (true) {
switch (c = in.read()) {
case -1:
case '\n':
break loop;
case '\r':
final int c2 = in.read();
if ((c2 != '\n') && (c2 != -1)) {
if (!(in instanceof PushbackInputStream)) {
in = new PushbackInputStream(in);
}
((PushbackInputStream) in).unread(c2);
}
break loop;
default:
if (--room < 0) {
buf = new char[offset + 128];
room = buf.length - offset - 1;
System.arraycopy(lineCache, 0, buf, 0, offset);
lineCache = buf;
}
buf[offset++] = (char) c;
break;
}
}
if ((c == -1) && (offset == 0)) {
return null;
}
return String.copyValueOf(buf, 0, offset);
}
@Override
public final String readUTF() throws IOException {
return readUTF(this);
}
private final static String readUTF(DataInput in) throws IOException {
final int utflen = in.readUnsignedShort();
byte[] bytearr = null;
char[] chararr = null;
bytearr = new byte[utflen];
chararr = new char[utflen];
int c, char2, char3;
int count = 0;
int chararr_count = 0;
in.readFully(bytearr, 0, utflen);
while (count < utflen) {
c = bytearr[count] & 0xff;
if (c > 127) {
break;
}
count++;
chararr[chararr_count++] = (char) c;
}
while (count < utflen) {
c = bytearr[count] & 0xff;
switch (c >> 4) {
case 0:
case 1:
case 2:
case 3:
case 4:
case 5:
case 6:
case 7:
/* 0xxxxxxx */
count++;
chararr[chararr_count++] = (char) c;
break;
case 12:
case 13:
/* 110x xxxx 10xx xxxx */
count += 2;
if (count > utflen) {
throw new UTFDataFormatException("malformed input: partial character at end");
}
char2 = bytearr[count - 1];
if ((char2 & 0xC0) != 0x80) {
throw new UTFDataFormatException("malformed input around byte " + count);
}
chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
break;
case 14:
/* 1110 xxxx 10xx xxxx 10xx xxxx */
count += 3;
if (count > utflen) {
throw new UTFDataFormatException("malformed input: partial character at end");
}
char2 = bytearr[count - 2];
char3 = bytearr[count - 1];
if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
throw new UTFDataFormatException("malformed input around byte " + (count - 1));
}
chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
break;
default:
/* 10xx xxxx, 1111 xxxx */
throw new UTFDataFormatException("malformed input around byte " + count);
}
}
// The number of chars produced may be less than utflen
return new String(chararr, 0, chararr_count);
}
@Override
public void close() throws IOException {
super.close();
}
@Override
public boolean hasUnReadData() {
return null != byteBuffer && byteBuffer.hasRemaining();
}
}

View File

@ -0,0 +1,252 @@
/**
* 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.UTFDataFormatException;
import java.nio.ByteBuffer;
import org.apache.hadoop.mapred.nativetask.NativeDataTarget;
/**
* write data to a output buffer
*/
public class ByteBufferDataWriter extends DataOutputStream {
private ByteBuffer buffer;
private final NativeDataTarget target;
private void checkSizeAndFlushNecessary(int length) throws IOException {
if (buffer.position() > 0 && buffer.remaining() < length) {
flush();
}
}
public ByteBufferDataWriter(NativeDataTarget handler) {
if (null != handler) {
this.buffer = handler.getOutputBuffer().getByteBuffer();
}
this.target = handler;
}
@Override
public synchronized void write(int v) throws IOException {
checkSizeAndFlushNecessary(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();
}
private final static byte TRUE = (byte) 1;
private final static byte FALSE = (byte) 0;
@Override
public final void writeBoolean(boolean v) throws IOException {
checkSizeAndFlushNecessary(1);
buffer.put(v ? TRUE : FALSE);
}
@Override
public final void writeByte(int v) throws IOException {
checkSizeAndFlushNecessary(1);
buffer.put((byte) v);
}
@Override
public final void writeShort(int v) throws IOException {
checkSizeAndFlushNecessary(2);
buffer.putShort((short) v);
}
@Override
public final void writeChar(int v) throws IOException {
checkSizeAndFlushNecessary(2);
buffer.put((byte) ((v >>> 8) & 0xFF));
buffer.put((byte) ((v >>> 0) & 0xFF));
}
@Override
public final void writeInt(int v) throws IOException {
checkSizeAndFlushNecessary(4);
buffer.putInt(v);
}
@Override
public final void writeLong(long v) throws IOException {
checkSizeAndFlushNecessary(8);
buffer.putLong(v);
}
@Override
public final void writeFloat(float v) throws IOException {
checkSizeAndFlushNecessary(4);
writeInt(Float.floatToIntBits(v));
}
@Override
public final void writeDouble(double v) throws IOException {
checkSizeAndFlushNecessary(8);
writeLong(Double.doubleToLongBits(v));
}
@Override
public final void writeBytes(String s) throws IOException {
final int len = s.length();
int remain = len;
int offset = 0;
while (remain > 0) {
int currentFlush = 0;
if (buffer.remaining() > 0) {
currentFlush = Math.min(buffer.remaining(), remain);
for (int i = 0; i < currentFlush; i++) {
buffer.put((byte) s.charAt(offset + i));
}
remain -= currentFlush;
offset += currentFlush;
} else {
flush();
}
}
}
@Override
public final void writeChars(String s) throws IOException {
final int len = s.length();
int remain = len;
int offset = 0;
while (remain > 0) {
int currentFlush = 0;
if (buffer.remaining() > 2) {
currentFlush = Math.min(buffer.remaining() / 2, remain);
for (int i = 0; i < currentFlush; i++) {
buffer.putChar(s.charAt(offset + i));
}
remain -= currentFlush;
offset += currentFlush;
} else {
flush();
}
}
}
@Override
public final void writeUTF(String str) throws IOException {
writeUTF(str, this);
}
private int writeUTF(String str, DataOutput out) throws IOException {
final int strlen = str.length();
int utflen = 0;
int c, count = 0;
/* use charAt instead of copying String to char array */
for (int i = 0; i < strlen; i++) {
c = str.charAt(i);
if ((c >= 0x0001) && (c <= 0x007F)) {
utflen++;
} else if (c > 0x07FF) {
utflen += 3;
} else {
utflen += 2;
}
}
if (utflen > 65535) {
throw new UTFDataFormatException("encoded string too long: " + utflen + " bytes");
}
final byte[] bytearr = new byte[utflen + 2];
bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF);
bytearr[count++] = (byte) ((utflen >>> 0) & 0xFF);
int i = 0;
for (i = 0; i < strlen; i++) {
c = str.charAt(i);
if (!((c >= 0x0001) && (c <= 0x007F))) {
break;
}
bytearr[count++] = (byte) c;
}
for (; i < strlen; i++) {
c = str.charAt(i);
if ((c >= 0x0001) && (c <= 0x007F)) {
bytearr[count++] = (byte) c;
} else if (c > 0x07FF) {
bytearr[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
bytearr[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
bytearr[count++] = (byte) (0x80 | ((c >> 0) & 0x3F));
} else {
bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
bytearr[count++] = (byte) (0x80 | ((c >> 0) & 0x3F));
}
}
write(bytearr, 0, utflen + 2);
return utflen + 2;
}
@Override
public boolean hasUnFlushedData() {
return !(buffer.position() == 0);
}
}

View File

@ -0,0 +1,25 @@
/**
* 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;
public abstract class DataInputStream extends InputStream implements DataInput {
public abstract boolean hasUnReadData();
}

View File

@ -0,0 +1,42 @@
/**
* 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;
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
* @return
* @throws IOException
*/
public abstract boolean shortOfSpace(int length) throws IOException;
/**
* Check whether there is unflushed data stored in the stream
*
* @return
*/
public abstract boolean hasUnFlushedData();
}

View File

@ -0,0 +1,93 @@
/**
* 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.lang.ref.WeakReference;
import java.nio.ByteBuffer;
import java.util.Queue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ConcurrentMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
/**
* as direct buffer memory is not collected by GC, we keep a pool
* to reuse direct buffers
*/
public class DirectBufferPool {
private static DirectBufferPool directBufferPool = null;
private static Log LOG = LogFactory.getLog(DirectBufferPool.class);
private ConcurrentMap<Integer, Queue<WeakReference<ByteBuffer>>> bufferMap = new ConcurrentHashMap<Integer, Queue<WeakReference<ByteBuffer>>>();
private DirectBufferPool() {
}
public static synchronized DirectBufferPool getInstance() {
if (null == directBufferPool) {
directBufferPool = new DirectBufferPool();
}
return directBufferPool;
}
public static void destoryInstance(){
directBufferPool = null;
}
public synchronized ByteBuffer borrowBuffer(int capacity) throws IOException {
Queue<WeakReference<ByteBuffer>> list = bufferMap.get(capacity);
if (null == list) {
return ByteBuffer.allocateDirect(capacity);
}
WeakReference<ByteBuffer> ref;
while ((ref = list.poll()) != null) {
ByteBuffer buf = ref.get();
if (buf != null) {
return buf;
}
}
return ByteBuffer.allocateDirect(capacity);
}
public void returnBuffer(ByteBuffer buffer) throws IOException {
if (null == buffer || !buffer.isDirect()) {
throw new IOException("the buffer is null or the buffer returned is not direct buffer");
}
buffer.clear();
int capacity = buffer.capacity();
Queue<WeakReference<ByteBuffer>> list = bufferMap.get(capacity);
if (null == list) {
list = new ConcurrentLinkedQueue<WeakReference<ByteBuffer>>();
Queue<WeakReference<ByteBuffer>> prev = bufferMap.putIfAbsent(capacity, list);
if (prev != null) {
list = prev;
}
}
list.add(new WeakReference<ByteBuffer>(buffer));
}
int getBufCountsForCapacity(int capacity) {
return bufferMap.get(capacity).size();
}
}

View File

@ -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.buffer;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
public class InputBuffer {
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 = DirectBufferPool.getInstance().borrowBuffer(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();
}
}

View File

@ -0,0 +1,74 @@
/**
* 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;
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();
}
}

View File

@ -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.
*/
package org.apache.hadoop.mapred.nativetask.handlers;
import java.io.IOException;
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}
*/
public class BufferPullee<IK, IV> implements IDataLoader {
public static 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;
}
}

View File

@ -0,0 +1,187 @@
package org.apache.hadoop.mapred.nativetask.handlers;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
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.mapred.nativetask.serde.SerializationFramework;
import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer;
import org.apache.hadoop.util.Progress;
/**
* actively signal a {@link BufferPullee} to load data into buffer and receive
*/
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];
}
nativeReader.read(keyBytes, 0, keyLength);
nativeReader.read(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.
*
* @throws IOException
*/
@Override
public void close() throws IOException {
if (closed) {
return;
}
if (null != nativeReader) {
nativeReader.close();
}
closed = true;
}
}

View File

@ -0,0 +1,147 @@
/**
* 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.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
*/
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;
}
}

View File

@ -0,0 +1,89 @@
/**
* 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.io.Text;
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
*/
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;
}
}

View File

@ -0,0 +1,140 @@
/**
* 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 static org.apache.hadoop.mapred.Task.Counter.COMBINE_INPUT_RECORDS;
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;
public class CombinerHandler<K, V> implements ICombineHandler, CommandDispatcher {
public static String NAME = "NativeTask.CombineHandler";
private static Log LOG = LogFactory.getLog(NativeCollectorOnlyHandler.class);
public static Command LOAD = new Command(1, "Load");
public static 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(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);
final BufferPusher<K, V> pusher = new BufferPusher<K, V>(context.getInputKeyClass(), 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;
}
}

View 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.handlers;
import java.io.IOException;
/**
* an IDataLoader loads data on demand
*/
public interface IDataLoader {
/**
* @return size of data loaded
* @throws IOException
*/
public int load() throws IOException;
public void close() throws IOException;
}

View File

@ -0,0 +1,161 @@
/**
* 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.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")
public class NativeCollectorOnlyHandler<K, V> implements CommandDispatcher, Closeable {
public static String NAME = "NativeTask.MCollectorOutputHandler";
private static Log LOG = LogFactory.getLog(NativeCollectorOnlyHandler.class);
public static Command GET_OUTPUT_PATH = new Command(100, "GET_OUTPUT_PATH");
public static Command GET_OUTPUT_INDEX_PATH = new Command(101, "GET_OUTPUT_INDEX_PATH");
public static Command GET_SPILL_PATH = new Command(102, "GET_SPILL_PATH");
public static 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.getOuputKeyClass());
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>(context.getOuputKeyClass(), 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");
}
}
}

View File

@ -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.serde;
import java.io.IOException;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.nativetask.INativeComparable;
public class BoolWritableSerializer extends DefaultSerializer implements
INativeComparable {
@Override
public int getLength(Writable w) throws IOException {
return 1;
}
}

View File

@ -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.serde;
import java.io.IOException;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.nativetask.INativeComparable;
public class ByteWritableSerializer extends DefaultSerializer implements
INativeComparable {
@Override
public int getLength(Writable w) throws IOException {
return 1;
}
}

View File

@ -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.
*/
package org.apache.hadoop.mapred.nativetask.serde;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.mapred.nativetask.INativeComparable;
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);
}
}

View File

@ -0,0 +1,67 @@
/**
* 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.io.Writable;
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);
}
}

View File

@ -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.serde;
import java.io.IOException;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.nativetask.INativeComparable;
public class DoubleWritableSerializer extends DefaultSerializer implements
INativeComparable {
@Override
public int getLength(Writable w) throws IOException {
return 8;
}
}

View File

@ -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.serde;
import java.io.IOException;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.nativetask.INativeComparable;
public class FloatWritableSerializer extends DefaultSerializer implements
INativeComparable {
@Override
public int getLength(Writable w) throws IOException {
return 4;
}
}

View File

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

View File

@ -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.
*/
package org.apache.hadoop.mapred.nativetask.serde;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
/**
* 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.
*
*/
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;
}

View File

@ -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.serde;
import java.io.IOException;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.nativetask.INativeComparable;
public class IntWritableSerializer extends DefaultSerializer implements
INativeComparable {
@Override
public int getLength(Writable w) throws IOException {
return 4;
}
}

View File

@ -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.
*/
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.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;
public class KVSerializer<K, V> implements IKVSerializer {
private static final Log LOG = LogFactory.getLog(KVSerializer.class);
public static 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;
}
}

View File

@ -0,0 +1,32 @@
/**
* 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.io.Writable;
import org.apache.hadoop.mapred.nativetask.INativeComparable;
public class LongWritableSerializer extends DefaultSerializer implements
INativeComparable {
@Override
public int getLength(Writable w) throws IOException {
return 8;
}
}

View File

@ -0,0 +1,86 @@
/**
* 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.io.Writable;
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, exist: " + storedSerializer.getName() + ", new: "
+ serializer.getName());
}
}
}
public void reset() {
map.clear();
}
private static NativeSerialization instance = new NativeSerialization();
public static NativeSerialization getInstance() {
return instance;
}
}

View File

@ -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.serde;
import java.io.IOException;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.nativetask.INativeComparable;
public class NullWritableSerializer extends DefaultSerializer implements
INativeComparable {
@Override
public int getLength(Writable w) throws IOException {
return 0;
}
}

View File

@ -0,0 +1,32 @@
/**
* 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;
public enum SerializationFramework {
WRITABLE_SERIALIZATION(0), NATIVE_SERIALIZATION(1);
private int type;
SerializationFramework(int type) {
this.type = type;
}
public int getType() {
return type;
}
};

View File

@ -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.serde;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.nativetask.INativeComparable;
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 {
try {
w.setCapacity(length, true);
w.setLength(length);
} catch (final Exception e) {
throw new IOException(e);
}
final byte[] bytes = w.getBytes();
in.readFully(bytes, 0, length);
}
}

View File

@ -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.
*/
package org.apache.hadoop.mapred.nativetask.serde;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.nativetask.INativeComparable;
public class VIntWritableSerializer extends DefaultSerializer implements
INativeComparable {
}

View File

@ -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.
*/
package org.apache.hadoop.mapred.nativetask.serde;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.nativetask.INativeComparable;
public class VLongWritableSerializer extends DefaultSerializer implements
INativeComparable {
}

View File

@ -0,0 +1,267 @@
/**
* 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.UnsupportedEncodingException;
public class BytesUtil {
public static final int SIZEOF_INT = Integer.SIZE / Byte.SIZE;
public static final int SIZEOF_LONG = Long.SIZE / Byte.SIZE;
public static byte[] toBytes(String str) {
if (str == null) {
return null;
}
try {
return str.getBytes("utf-8");
} catch (final UnsupportedEncodingException e) {
throw new RuntimeException(e.getMessage());
}
}
public static String fromBytes(byte[] data) {
if (data == null) {
return null;
}
try {
return new String(data, "utf-8");
} catch (final UnsupportedEncodingException e) {
throw new RuntimeException(e.getMessage());
}
}
/**
* Converts a byte array to an int value
* @param bytes byte array
* @return the int value
*/
public static int toInt(byte[] bytes) {
return toInt(bytes, 0, SIZEOF_INT);
}
/**
* Converts a byte array to an int value
* @param bytes byte array
* @param offset offset into array
* @param length length of int (has to be {@link #SIZEOF_INT})
* @return the int value
* @throws RuntimeException if length is not {@link #SIZEOF_INT} or
* if there's not enough room in the array at the offset indicated.
*/
public static int toInt(byte[] bytes, int offset, final int length) {
if (length != SIZEOF_INT || offset + length > bytes.length) {
throw new RuntimeException(
"toInt exception. length not equals to SIZE of Int or buffer overflow");
}
int n = 0;
for (int i = offset; i< offset + length; i++) {
n <<= 4;
n ^= bytes[i] & 0xff;
}
return n;
}
/**
* Converts a byte array to a long value.
* @param bytes array
* @return the long value
*/
public static long toLong(byte[] bytes) {
return toLong(bytes, 0, SIZEOF_LONG);
}
/**
* Converts a byte array to a long value.
*
* @param bytes array of bytes
* @param offset offset into array
* @return the long value
*/
public static long toLong(byte[] bytes, int offset) {
return toLong(bytes, offset, SIZEOF_LONG);
}
/**
* Converts a byte array to a long value.
*
* @param bytes array of bytes
* @param offset offset into array
* @param length length of data (must be {@link #SIZEOF_LONG})
* @return the long value
* @throws RuntimeException if length is not {@link #SIZEOF_LONG} or
* if there's not enough room in the array at the offset indicated.
*/
public static long toLong(byte[] bytes, int offset, final int length) {
if (length != SIZEOF_LONG || offset + length > bytes.length) {
throw new RuntimeException(
"toLong exception. length not equals to SIZE of Long or buffer overflow");
}
long l = 0;
for (int i = offset; i < offset + length; i++) {
l <<= 8;
l ^= bytes[i] & 0xff;
}
return l;
}
/**
* 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, SIZEOF_INT));
}
/**
* @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, SIZEOF_LONG));
}
/**
* Write a printable representation of a byte array.
*
* @param b byte array
* @return string
* @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(String.format("\\x%02X", ch));
}
}
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 };
}
/**
* Convert an int value to a byte array. Big-endian. Same as what DataOutputStream.writeInt
* does.
*
* @param val value
* @return the byte array
*/
public static byte[] toBytes(int val) {
byte [] b = new byte[4];
for(int i = 3; i > 0; i--) {
b[i] = (byte) val;
val >>>= 8;
}
b[0] = (byte) val;
return b;
}
/**
* Convert a long value to a byte array using big-endian.
*
* @param val value to convert
* @return the byte array
*/
public static byte[] toBytes(long val) {
byte [] b = new byte[8];
for (int i = 7; i > 0; i--) {
b[i] = (byte) val;
val >>>= 8;
}
b[0] = (byte) val;
return b;
}
/**
* @param f float value
* @return the float represented as byte []
*/
public static byte [] toBytes(final float f) {
// Encode it as int
return toBytes(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 toBytes(Double.doubleToRawLongBits(d));
}
}

View File

@ -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.
*/
package org.apache.hadoop.mapred.nativetask.util;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
public class ConfigUtil {
public static byte[][] toBytes(Configuration conf) {
List<byte[]> nativeConfigs = new ArrayList<byte[]>();
for (Map.Entry<String, String> e : conf) {
nativeConfigs.add(BytesUtil.toBytes(e.getKey()));
nativeConfigs.add(BytesUtil.toBytes(e.getValue()));
}
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();
}
}

View File

@ -0,0 +1,189 @@
/**
* 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.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;
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
*
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
*
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
public Path getInputFileForWrite(TaskID mapId, long size, Configuration conf) throws IOException {
return lDirAlloc.getLocalPathForWrite(
String.format(REDUCE_INPUT_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, mapId.getId()), size,
conf);
}
/** 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);
}
}

View File

@ -0,0 +1,137 @@
/**
* 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.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.TaskID;
/**
* base class of output files manager.
*/
public interface NativeTaskOutput {
/**
* Return the path to local map output file created earlier
*
* @return path
* @throws IOException
*/
public Path getOutputFile() throws IOException;
/**
* Create a local map output file name.
*
* @param size
* the size of the file
* @return path
* @throws IOException
*/
public Path getOutputFileForWrite(long size) throws IOException;
/**
* Return the path to a local map output index file created earlier
*
* @return path
* @throws IOException
*/
public Path getOutputIndexFile() throws IOException;
/**
* Create a local map output index file name.
*
* @param size
* the size of the file
* @return path
* @throws IOException
*/
public Path getOutputIndexFileForWrite(long size) throws IOException;
/**
* Return a local map spill file created earlier.
*
* @param spillNumber
* the number
* @return path
* @throws IOException
*/
public Path getSpillFile(int spillNumber) throws IOException;
/**
* Create a local map spill file name.
*
* @param spillNumber
* the number
* @param size
* the size of the file
* @return path
* @throws IOException
*/
public Path getSpillFileForWrite(int spillNumber, long size) throws IOException;
/**
* Return a local map spill index file created earlier
*
* @param spillNumber
* the number
* @return path
* @throws IOException
*/
public Path getSpillIndexFile(int spillNumber) throws IOException;
/**
* Create a local map spill index file name.
*
* @param spillNumber
* the number
* @param size
* the size of the file
* @return path
* @throws IOException
*/
public Path getSpillIndexFileForWrite(int spillNumber, long size) throws IOException;
/**
* Return a local reduce input file created earlier
*
* @param mapId
* a map task id
* @return path
* @throws IOException
*/
public Path getInputFile(int mapId) throws IOException;
/**
* Create a local reduce input file name.
*
* @param mapId
* a map task id
* @param size
* the size of the file
* @return path
* @throws IOException
*/
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);
}

View File

@ -0,0 +1,199 @@
/**
* 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.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.
*/
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
*
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
*
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
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
* @return path
* @throws IOException
*/
public Path getInputFileForWrite(TaskID mapId, long size, Configuration conf) throws IOException {
return lDirAlloc.getLocalPathForWrite(
String.format(REDUCE_INPUT_FILE_FORMAT_STRING, TASKTRACKER_OUTPUT, mapId.getId()), size,
conf);
}
/** 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);
}
}

View File

@ -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.
*/
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.conf.Configuration;
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) {
return null;
}
}
}

View File

@ -0,0 +1,150 @@
/**
* 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;
public class ReadWriteBuffer {
private byte[] _buff;
private int _writePoint;
private int _readPoint;
final 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 = BytesUtil.toBytes(str);
writeBytes(bytes, 0, bytes.length);
}
public String readString() {
final byte[] bytes = readBytes();
return BytesUtil.fromBytes(bytes);
}
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;
}
};

View File

@ -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.util;
import java.io.IOException;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.util.ReflectionUtils;
public class SizedWritable<T> {
public static 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;
}
}

View File

@ -0,0 +1,32 @@
/*
* 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 org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.io.compress.SnappyCodec;
public class SnappyUtil {
public static boolean isNativeSnappyLoaded(Configuration conf) {
return SnappyCodec.isNativeCodeLoaded() && conf.getBoolean(
CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY,
CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_DEFAULT);
}
}

View File

@ -0,0 +1,83 @@
CityHash
---------------------------------------------------------------------
// Copyright (c) 2011 Google, Inc.
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to deal
// in the Software without restriction, including without limitation the rights
// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
// copies of the Software, and to permit persons to whom the Software is
// furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
// THE SOFTWARE.
GoogleTest
---------------------------------------------------------------------
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.
LZ4
---------------------------------------------------------------------
LZ4 - Fast LZ compression algorithm
Header File
Copyright (C) 2011, Yann Collet.
BSD License
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.
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.

View File

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

View File

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

View File

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

View File

@ -0,0 +1,740 @@
/*
LZ4 - Fast LZ compression algorithm
Copyright (C) 2011-2012, Yann Collet.
BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
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.
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.
*/
//**************************************
// Compilation Directives
//**************************************
#if __STDC_VERSION__ >= 199901L
/* "restrict" is a known keyword */
#else
#define restrict // Disable restrict
#endif
#ifdef _MSC_VER
#define inline __forceinline
#endif
#ifdef __GNUC__
#define _PACKED __attribute__ ((packed))
#else
#define _PACKED
#endif
#if (__x86_64__ || __ppc64__ || _WIN64 || __LP64__) // Detect 64 bits mode
#define ARCH64 1
#else
#define ARCH64 0
#endif
//**************************************
// Includes
//**************************************
#include <stdlib.h> // for malloc
#include <string.h> // for memset
#include "lz4.h"
//**************************************
// Performance parameter
//**************************************
// Increasing this value improves compression ratio
// Lowering this value reduces memory usage
// Lowering may also improve speed, typically on reaching cache size limits (L1 32KB for Intel, 64KB for AMD)
// Memory usage formula for 32 bits systems : N->2^(N+2) Bytes (examples : 17 -> 512KB ; 12 -> 16KB)
#define HASH_LOG 12
//#define _FORCE_SW_BITCOUNT // Uncomment for better performance if target platform has no hardware support for LowBitCount
//**************************************
// Basic Types
//**************************************
#if defined(_MSC_VER) // Visual Studio does not support 'stdint' natively
#define BYTE unsigned __int8
#define U16 unsigned __int16
#define U32 unsigned __int32
#define S32 __int32
#define U64 unsigned __int64
#else
#include <stdint.h>
#define BYTE uint8_t
#define U16 uint16_t
#define U32 uint32_t
#define S32 int32_t
#define U64 uint64_t
#endif
//**************************************
// Constants
//**************************************
#define MINMATCH 4
#define SKIPSTRENGTH 6
#define STACKLIMIT 13
#define HEAPMODE (HASH_LOG>STACKLIMIT) // Defines if memory is allocated into the stack (local variable), or into the heap (malloc()).
#define COPYLENGTH 8
#define LASTLITERALS 5
#define MFLIMIT (COPYLENGTH+MINMATCH)
#define MINLENGTH (MFLIMIT+1)
#define MAXD_LOG 16
#define MAX_DISTANCE ((1 << MAXD_LOG) - 1)
#define HASHTABLESIZE (1 << HASH_LOG)
#define HASH_MASK (HASHTABLESIZE - 1)
#define ML_BITS 4
#define ML_MASK ((1U<<ML_BITS)-1)
#define RUN_BITS (8-ML_BITS)
#define RUN_MASK ((1U<<RUN_BITS)-1)
//**************************************
// Local structures
//**************************************
struct refTables
{
const BYTE* hashTable[HASHTABLESIZE];
};
typedef struct _U64_S
{
U64 v;
} _PACKED U64_S;
typedef struct _U32_S
{
U32 v;
} _PACKED U32_S;
typedef struct _U16_S
{
U16 v;
} _PACKED U16_S;
#define A64(x) (((U64_S *)(x))->v)
#define A32(x) (((U32_S *)(x))->v)
#define A16(x) (((U16_S *)(x))->v)
//**************************************
// Architecture-specific macros
//**************************************
#if ARCH64 // 64-bit
#define STEPSIZE 8
#define UARCH U64
#define AARCH A64
#define LZ4_COPYSTEP(s,d) A64(d) = A64(s); d+=8; s+=8;
#define LZ4_COPYPACKET(s,d) LZ4_COPYSTEP(s,d)
#define LZ4_SECURECOPY(s,d,e) if (d<e) LZ4_WILDCOPY(s,d,e)
#define HTYPE U32
#define INITBASE(base) const BYTE* const base = ip
#else // 32-bit
#define STEPSIZE 4
#define UARCH U32
#define AARCH A32
#define LZ4_COPYSTEP(s,d) A32(d) = A32(s); d+=4; s+=4;
#define LZ4_COPYPACKET(s,d) LZ4_COPYSTEP(s,d); LZ4_COPYSTEP(s,d);
#define LZ4_SECURECOPY LZ4_WILDCOPY
#define HTYPE const BYTE*
#define INITBASE(base) const int base = 0
#endif
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
#define LZ4_READ_LITTLEENDIAN_16(d,s,p) { d = (s) - A16(p); }
#define LZ4_WRITE_LITTLEENDIAN_16(p,v) { A16(p) = v; p+=2; }
#define LZ4_NbCommonBytes LZ4_NbCommonBytes_LittleEndian
#else // Big Endian
#define LZ4_READ_LITTLEENDIAN_16(d,s,p) { int delta = p[0]; delta += p[1] << 8; d = (s) - delta; }
#define LZ4_WRITE_LITTLEENDIAN_16(p,v) { int delta = v; *p++ = delta; *p++ = delta>>8; }
#define LZ4_NbCommonBytes LZ4_NbCommonBytes_BigEndian
#endif
//**************************************
// Macros
//**************************************
#define LZ4_HASH_FUNCTION(i) (((i) * 2654435761U) >> ((MINMATCH*8)-HASH_LOG))
#define LZ4_HASH_VALUE(p) LZ4_HASH_FUNCTION(A32(p))
#define LZ4_WILDCOPY(s,d,e) do { LZ4_COPYPACKET(s,d) } while (d<e);
#define LZ4_BLINDCOPY(s,d,l) { BYTE* e=(d)+l; LZ4_WILDCOPY(s,d,e); d=e; }
//****************************
// Private functions
//****************************
#if ARCH64
inline static int LZ4_NbCommonBytes_LittleEndian (register U64 val)
{
#if defined(_MSC_VER) && !defined(_FORCE_SW_BITCOUNT)
unsigned long r = 0;
_BitScanForward64( &r, val );
return (int)(r>>3);
#elif defined(__GNUC__) && !defined(_FORCE_SW_BITCOUNT)
return (__builtin_ctzll(val) >> 3);
#else
static const int DeBruijnBytePos[64] = { 0, 0, 0, 0, 0, 1, 1, 2, 0, 3, 1, 3, 1, 4, 2, 7, 0, 2, 3, 6, 1, 5, 3, 5, 1, 3, 4, 4, 2, 5, 6, 7, 7, 0, 1, 2, 3, 3, 4, 6, 2, 6, 5, 5, 3, 4, 5, 6, 7, 1, 2, 4, 6, 4, 4, 5, 7, 2, 6, 5, 7, 6, 7, 7 };
return DeBruijnBytePos[((U64)((val & -val) * 0x0218A392CDABBD3F)) >> 58];
#endif
}
inline static int LZ4_NbCommonBytes_BigEndian (register U64 val)
{
#if defined(_MSC_VER) && !defined(_FORCE_SW_BITCOUNT)
unsigned long r = 0;
_BitScanReverse64( &r, val );
return (int)(r>>3);
#elif defined(__GNUC__) && !defined(_FORCE_SW_BITCOUNT)
return (__builtin_clzll(val) >> 3);
#else
int r;
if (!(val>>32)) { r=4; } else { r=0; val>>=32; }
if (!(val>>16)) { r+=2; val>>=8; } else { val>>=24; }
r += (!val);
return r;
#endif
}
#else
inline static int LZ4_NbCommonBytes_LittleEndian (register U32 val)
{
#if defined(_MSC_VER) && !defined(_FORCE_SW_BITCOUNT)
unsigned long r = 0;
_BitScanForward( &r, val );
return (int)(r>>3);
#elif defined(__GNUC__) && !defined(_FORCE_SW_BITCOUNT)
return (__builtin_ctz(val) >> 3);
#else
static const int DeBruijnBytePos[32] = { 0, 0, 3, 0, 3, 1, 3, 0, 3, 2, 2, 1, 3, 2, 0, 1, 3, 3, 1, 2, 2, 2, 2, 0, 3, 1, 2, 0, 1, 0, 1, 1 };
return DeBruijnBytePos[((U32)((val & -val) * 0x077CB531U)) >> 27];
#endif
}
inline static int LZ4_NbCommonBytes_BigEndian (register U32 val)
{
#if defined(_MSC_VER) && !defined(_FORCE_SW_BITCOUNT)
unsigned long r = 0;
_BitScanReverse( &r, val );
return (int)(r>>3);
#elif defined(__GNUC__) && !defined(_FORCE_SW_BITCOUNT)
return (__builtin_clz(val) >> 3);
#else
int r;
if (!(val>>16)) { r=2; val>>=8; } else { r=0; val>>=24; }
r += (!val);
return r;
#endif
}
#endif
//******************************
// Public Compression functions
//******************************
int LZ4_compressCtx(void** ctx,
char* source,
char* dest,
int isize)
{
#if HEAPMODE
struct refTables *srt = (struct refTables *) (*ctx);
HTYPE* HashTable;
#else
HTYPE HashTable[HASHTABLESIZE] = {0};
#endif
const BYTE* ip = (BYTE*) source;
INITBASE(base);
const BYTE* anchor = ip;
const BYTE* const iend = ip + isize;
const BYTE* const mflimit = iend - MFLIMIT;
#define matchlimit (iend - LASTLITERALS)
BYTE* op = (BYTE*) dest;
int len, length;
const int skipStrength = SKIPSTRENGTH;
U32 forwardH;
// Init
if (isize<MINLENGTH) goto _last_literals;
#if HEAPMODE
if (*ctx == NULL)
{
srt = (struct refTables *) malloc ( sizeof(struct refTables) );
*ctx = (void*) srt;
}
HashTable = (HTYPE*)(srt->hashTable);
memset((void*)HashTable, 0, sizeof(srt->hashTable));
#else
(void) ctx;
#endif
// First Byte
HashTable[LZ4_HASH_VALUE(ip)] = ip - base;
ip++; forwardH = LZ4_HASH_VALUE(ip);
// Main Loop
for ( ; ; )
{
int findMatchAttempts = (1U << skipStrength) + 3;
const BYTE* forwardIp = ip;
const BYTE* ref;
BYTE* token;
// Find a match
do {
U32 h = forwardH;
int step = findMatchAttempts++ >> skipStrength;
ip = forwardIp;
forwardIp = ip + step;
if (forwardIp > mflimit) { goto _last_literals; }
forwardH = LZ4_HASH_VALUE(forwardIp);
ref = base + HashTable[h];
HashTable[h] = ip - base;
} while ((ref < ip - MAX_DISTANCE) || (A32(ref) != A32(ip)));
// Catch up
while ((ip>anchor) && (ref>(BYTE*)source) && (ip[-1]==ref[-1])) { ip--; ref--; }
// Encode Literal length
length = ip - anchor;
token = op++;
if (length>=(int)RUN_MASK) { *token=(RUN_MASK<<ML_BITS); len = length-RUN_MASK; for(; len > 254 ; len-=255) *op++ = 255; *op++ = (BYTE)len; }
else *token = (length<<ML_BITS);
// Copy Literals
LZ4_BLINDCOPY(anchor, op, length);
_next_match:
// Encode Offset
LZ4_WRITE_LITTLEENDIAN_16(op,ip-ref);
// Start Counting
ip+=MINMATCH; ref+=MINMATCH; // MinMatch verified
anchor = ip;
while (ip<matchlimit-(STEPSIZE-1))
{
UARCH diff = AARCH(ref) ^ AARCH(ip);
if (!diff) { ip+=STEPSIZE; ref+=STEPSIZE; continue; }
ip += LZ4_NbCommonBytes(diff);
goto _endCount;
}
if (ARCH64) if ((ip<(matchlimit-3)) && (A32(ref) == A32(ip))) { ip+=4; ref+=4; }
if ((ip<(matchlimit-1)) && (A16(ref) == A16(ip))) { ip+=2; ref+=2; }
if ((ip<matchlimit) && (*ref == *ip)) ip++;
_endCount:
// Encode MatchLength
len = (ip - anchor);
if (len>=(int)ML_MASK) { *token+=ML_MASK; len-=ML_MASK; for(; len > 509 ; len-=510) { *op++ = 255; *op++ = 255; } if (len > 254) { len-=255; *op++ = 255; } *op++ = (BYTE)len; }
else *token += len;
// Test end of chunk
if (ip > mflimit) { anchor = ip; break; }
// Fill table
HashTable[LZ4_HASH_VALUE(ip-2)] = ip - 2 - base;
// Test next position
ref = base + HashTable[LZ4_HASH_VALUE(ip)];
HashTable[LZ4_HASH_VALUE(ip)] = ip - base;
if ((ref > ip - (MAX_DISTANCE + 1)) && (A32(ref) == A32(ip))) { token = op++; *token=0; goto _next_match; }
// Prepare next loop
anchor = ip++;
forwardH = LZ4_HASH_VALUE(ip);
}
_last_literals:
// Encode Last Literals
{
int lastRun = iend - anchor;
if (lastRun>=(int)RUN_MASK) { *op++=(RUN_MASK<<ML_BITS); lastRun-=RUN_MASK; for(; lastRun > 254 ; lastRun-=255) *op++ = 255; *op++ = (BYTE) lastRun; }
else *op++ = (lastRun<<ML_BITS);
memcpy(op, anchor, iend - anchor);
op += iend-anchor;
}
// End
return (int) (((char*)op)-dest);
}
// Note : this function is valid only if isize < LZ4_64KLIMIT
#define LZ4_64KLIMIT ((1<<16) + (MFLIMIT-1))
#define HASHLOG64K (HASH_LOG+1)
#define HASH64KTABLESIZE (1U<<HASHLOG64K)
#define LZ4_HASH64K_FUNCTION(i) (((i) * 2654435761U) >> ((MINMATCH*8)-HASHLOG64K))
#define LZ4_HASH64K_VALUE(p) LZ4_HASH64K_FUNCTION(A32(p))
int LZ4_compress64kCtx(void** ctx,
char* source,
char* dest,
int isize)
{
#if HEAPMODE
struct refTables *srt = (struct refTables *) (*ctx);
U16* HashTable;
#else
U16 HashTable[HASH64KTABLESIZE] = {0};
#endif
const BYTE* ip = (BYTE*) source;
const BYTE* anchor = ip;
const BYTE* const base = ip;
const BYTE* const iend = ip + isize;
const BYTE* const mflimit = iend - MFLIMIT;
#define matchlimit (iend - LASTLITERALS)
BYTE* op = (BYTE*) dest;
int len, length;
const int skipStrength = SKIPSTRENGTH;
U32 forwardH;
// Init
if (isize<MINLENGTH) goto _last_literals;
#if HEAPMODE
if (*ctx == NULL)
{
srt = (struct refTables *) malloc ( sizeof(struct refTables) );
*ctx = (void*) srt;
}
HashTable = (U16*)(srt->hashTable);
memset((void*)HashTable, 0, sizeof(srt->hashTable));
#else
(void) ctx;
#endif
// First Byte
ip++; forwardH = LZ4_HASH64K_VALUE(ip);
// Main Loop
for ( ; ; )
{
int findMatchAttempts = (1U << skipStrength) + 3;
const BYTE* forwardIp = ip;
const BYTE* ref;
BYTE* token;
// Find a match
do {
U32 h = forwardH;
int step = findMatchAttempts++ >> skipStrength;
ip = forwardIp;
forwardIp = ip + step;
if (forwardIp > mflimit) { goto _last_literals; }
forwardH = LZ4_HASH64K_VALUE(forwardIp);
ref = base + HashTable[h];
HashTable[h] = ip - base;
} while (A32(ref) != A32(ip));
// Catch up
while ((ip>anchor) && (ref>(BYTE*)source) && (ip[-1]==ref[-1])) { ip--; ref--; }
// Encode Literal length
length = ip - anchor;
token = op++;
if (length>=(int)RUN_MASK) { *token=(RUN_MASK<<ML_BITS); len = length-RUN_MASK; for(; len > 254 ; len-=255) *op++ = 255; *op++ = (BYTE)len; }
else *token = (length<<ML_BITS);
// Copy Literals
LZ4_BLINDCOPY(anchor, op, length);
_next_match:
// Encode Offset
LZ4_WRITE_LITTLEENDIAN_16(op,ip-ref);
// Start Counting
ip+=MINMATCH; ref+=MINMATCH; // MinMatch verified
anchor = ip;
while (ip<matchlimit-(STEPSIZE-1))
{
UARCH diff = AARCH(ref) ^ AARCH(ip);
if (!diff) { ip+=STEPSIZE; ref+=STEPSIZE; continue; }
ip += LZ4_NbCommonBytes(diff);
goto _endCount;
}
if (ARCH64) if ((ip<(matchlimit-3)) && (A32(ref) == A32(ip))) { ip+=4; ref+=4; }
if ((ip<(matchlimit-1)) && (A16(ref) == A16(ip))) { ip+=2; ref+=2; }
if ((ip<matchlimit) && (*ref == *ip)) ip++;
_endCount:
// Encode MatchLength
len = (ip - anchor);
if (len>=(int)ML_MASK) { *token+=ML_MASK; len-=ML_MASK; for(; len > 509 ; len-=510) { *op++ = 255; *op++ = 255; } if (len > 254) { len-=255; *op++ = 255; } *op++ = (BYTE)len; }
else *token += len;
// Test end of chunk
if (ip > mflimit) { anchor = ip; break; }
// Fill table
HashTable[LZ4_HASH64K_VALUE(ip-2)] = ip - 2 - base;
// Test next position
ref = base + HashTable[LZ4_HASH64K_VALUE(ip)];
HashTable[LZ4_HASH64K_VALUE(ip)] = ip - base;
if (A32(ref) == A32(ip)) { token = op++; *token=0; goto _next_match; }
// Prepare next loop
anchor = ip++;
forwardH = LZ4_HASH64K_VALUE(ip);
}
_last_literals:
// Encode Last Literals
{
int lastRun = iend - anchor;
if (lastRun>=(int)RUN_MASK) { *op++=(RUN_MASK<<ML_BITS); lastRun-=RUN_MASK; for(; lastRun > 254 ; lastRun-=255) *op++ = 255; *op++ = (BYTE) lastRun; }
else *op++ = (lastRun<<ML_BITS);
memcpy(op, anchor, iend - anchor);
op += iend-anchor;
}
// End
return (int) (((char*)op)-dest);
}
int LZ4_compress(char* source,
char* dest,
int isize)
{
#if HEAPMODE
void* ctx = malloc(sizeof(struct refTables));
int result;
if (isize < LZ4_64KLIMIT)
result = LZ4_compress64kCtx(&ctx, source, dest, isize);
else result = LZ4_compressCtx(&ctx, source, dest, isize);
free(ctx);
return result;
#else
if (isize < (int)LZ4_64KLIMIT) return LZ4_compress64kCtx(NULL, source, dest, isize);
return LZ4_compressCtx(NULL, source, dest, isize);
#endif
}
//****************************
// Decompression functions
//****************************
// Note : The decoding functions LZ4_uncompress() and LZ4_uncompress_unknownOutputSize()
// are safe against "buffer overflow" attack type.
// They will never write nor read outside of the provided input and output buffers.
// A corrupted input will produce an error result, a negative int, indicating the position of the error within input stream.
int LZ4_uncompress(char* source,
char* dest,
int osize)
{
// Local Variables
const BYTE* restrict ip = (const BYTE*) source;
const BYTE* restrict ref;
BYTE* restrict op = (BYTE*) dest;
BYTE* const oend = op + osize;
BYTE* cpy;
BYTE token;
int len, length;
size_t dec[] ={0, 3, 2, 3, 0, 0, 0, 0};
// Main Loop
while (1)
{
// get runlength
token = *ip++;
if ((length=(token>>ML_BITS)) == RUN_MASK) { for (;(len=*ip++)==255;length+=255){} length += len; }
// copy literals
cpy = op+length;
if (cpy>oend-COPYLENGTH)
{
if (cpy > oend) goto _output_error;
memcpy(op, ip, length);
ip += length;
break; // Necessarily EOF
}
LZ4_WILDCOPY(ip, op, cpy); ip -= (op-cpy); op = cpy;
// get offset
LZ4_READ_LITTLEENDIAN_16(ref,cpy,ip); ip+=2;
if (ref < (BYTE* const)dest) goto _output_error;
// get matchlength
if ((length=(token&ML_MASK)) == ML_MASK) { for (;*ip==255;length+=255) {ip++;} length += *ip++; }
// copy repeated sequence
if (op-ref<STEPSIZE)
{
#if ARCH64
size_t dec2table[]={0, 4, 4, 3, 4, 5, 6, 7};
size_t dec2 = dec2table[op-ref];
#else
const int dec2 = 0;
#endif
*op++ = *ref++;
*op++ = *ref++;
*op++ = *ref++;
*op++ = *ref++;
ref -= dec[op-ref];
A32(op)=A32(ref); op += STEPSIZE-4; ref += STEPSIZE-4;
ref -= dec2;
} else { LZ4_COPYSTEP(ref,op); }
cpy = op + length - (STEPSIZE-4);
if (cpy>oend-COPYLENGTH)
{
if (cpy > oend) goto _output_error;
LZ4_SECURECOPY(ref, op, (oend-COPYLENGTH));
while(op<cpy) *op++=*ref++;
op=cpy;
if (op == oend) break; // Check EOF (should never happen, since last 5 bytes are supposed to be literals)
continue;
}
LZ4_SECURECOPY(ref, op, cpy);
op=cpy; // correction
}
// end of decoding
return (int) (((char*)ip)-source);
// write overflow error detected
_output_error:
return (int) (-(((char*)ip)-source));
}
int LZ4_uncompress_unknownOutputSize(
char* source,
char* dest,
int isize,
int maxOutputSize)
{
// Local Variables
const BYTE* restrict ip = (const BYTE*) source;
const BYTE* const iend = ip + isize;
const BYTE* restrict ref;
BYTE* restrict op = (BYTE*) dest;
BYTE* const oend = op + maxOutputSize;
BYTE* cpy;
BYTE token;
int len, length;
size_t dec[] ={0, 3, 2, 3, 0, 0, 0, 0};
// Main Loop
while (ip<iend)
{
// get runlength
token = *ip++;
if ((length=(token>>ML_BITS)) == RUN_MASK) { for (;(len=*ip++)==255;length+=255){} length += len; }
// copy literals
cpy = op+length;
if (cpy>oend-COPYLENGTH)
{
if (cpy > oend) goto _output_error;
memcpy(op, ip, length);
op += length;
break; // Necessarily EOF
}
LZ4_WILDCOPY(ip, op, cpy); ip -= (op-cpy); op = cpy;
if (ip>=iend) break; // check EOF
// get offset
LZ4_READ_LITTLEENDIAN_16(ref,cpy,ip); ip+=2;
if (ref < (BYTE* const)dest) goto _output_error;
// get matchlength
if ((length=(token&ML_MASK)) == ML_MASK) { for (;(len=*ip++)==255;length+=255){} length += len; }
// copy repeated sequence
if (op-ref<STEPSIZE)
{
#if ARCH64
size_t dec2table[]={0, 4, 4, 3, 4, 5, 6, 7};
size_t dec2 = dec2table[op-ref];
#else
const int dec2 = 0;
#endif
*op++ = *ref++;
*op++ = *ref++;
*op++ = *ref++;
*op++ = *ref++;
ref -= dec[op-ref];
A32(op)=A32(ref); op += STEPSIZE-4; ref += STEPSIZE-4;
ref -= dec2;
} else { LZ4_COPYSTEP(ref,op); }
cpy = op + length - (STEPSIZE-4);
if (cpy>oend-COPYLENGTH)
{
if (cpy > oend) goto _output_error;
LZ4_SECURECOPY(ref, op, (oend-COPYLENGTH));
while(op<cpy) *op++=*ref++;
op=cpy;
if (op == oend) break; // Check EOF (should never happen, since last 5 bytes are supposed to be literals)
continue;
}
LZ4_SECURECOPY(ref, op, cpy);
op=cpy; // correction
}
// end of decoding
return (int) (((char*)op)-dest);
// write overflow error detected
_output_error:
return (int) (-(((char*)ip)-source));
}

View File

@ -0,0 +1,96 @@
/*
LZ4 - Fast LZ compression algorithm
Header File
Copyright (C) 2011, Yann Collet.
BSD License
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.
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.
*/
#pragma once
#if defined (__cplusplus)
extern "C" {
#endif
//****************************
// Simple Functions
//****************************
int LZ4_compress (char* source, char* dest, int isize);
int LZ4_uncompress (char* source, char* dest, int osize);
/*
LZ4_compress() :
return : the number of bytes in compressed buffer dest
note : destination buffer must be already allocated.
To avoid any problem, size it to handle worst cases situations (input data not compressible)
Worst case size is : "inputsize + 0.4%", with "0.4%" being at least 8 bytes.
LZ4_uncompress() :
osize : is the output size, therefore the original size
return : the number of bytes read in the source buffer
If the source stream is malformed, the function will stop decoding and return a negative result, indicating the byte position of the faulty instruction
This version never writes beyond dest + osize, and is therefore protected against malicious data packets
note 2 : destination buffer must be already allocated
*/
//****************************
// Advanced Functions
//****************************
int LZ4_uncompress_unknownOutputSize (char* source, char* dest, int isize, int maxOutputSize);
/*
LZ4_uncompress_unknownOutputSize() :
isize : is the input size, therefore the compressed size
maxOutputSize : is the size of the destination buffer (which must be already allocated)
return : the number of bytes decoded in the destination buffer (necessarily <= maxOutputSize)
If the source stream is malformed, the function will stop decoding and return a negative result, indicating the byte position of the faulty instruction
This version never writes beyond dest + maxOutputSize, and is therefore protected against malicious data packets
note : This version is a bit slower than LZ4_uncompress
*/
int LZ4_compressCtx(void** ctx, char* source, char* dest, int isize);
/*
LZ4_compressCtx() :
This function explicitly handles the CTX memory structure.
It avoids allocating/deallocating memory between each call, improving performance when malloc is time-consuming.
Note : when memory is allocated into the stack (default mode), there is no "malloc" penalty.
Therefore, this function is mostly useful when memory is allocated into the heap (it requires increasing HASH_LOG value beyond STACK_LIMIT)
On first call : provide a *ctx=NULL; It will be automatically allocated.
On next calls : reuse the same ctx pointer.
Use different pointers for different threads when doing multi-threading.
note : performance difference is small, mostly noticeable in HeapMode when repetitively calling the compression function over many small segments.
*/
#if defined (__cplusplus)
}
#endif

View File

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

View File

@ -0,0 +1,180 @@
/**
* 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 "commons.h"
#include "NativeTask.h"
#include "BlockCodec.h"
namespace NativeTask {
BlockCompressStream::BlockCompressStream(OutputStream * stream, uint32_t bufferSizeHint)
: CompressStream(stream), _compressedBytesWritten(0), _tempBufferSize(0), _tempBuffer(NULL) {
_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), _tempBufferSize(0), _tempBuffer(NULL) {
_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

View File

@ -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 "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_ */

View File

@ -0,0 +1,199 @@
/**
* 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 "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;
std::cout << "gzip capacity " << _capacity << std::endl;
_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) {
free(_zstream);
_zstream = NULL;
}
delete[] _buffer;
_buffer = NULL;
}
void GzipCompressStream::write(const void * buff, uint32_t length) {
std::cout << "gzip " << length << std::endl;
z_stream * zstream = (z_stream*)_zstream;
zstream->next_in = (Bytef*)buff;
zstream->avail_in = length;
while (true) {
int ret = deflate(zstream, Z_NO_FLUSH);
std::cout << "gzip ret status " << ret << std::endl;
if (ret == Z_OK) {
std::cout << "gzip avail_out " << zstream->avail_out << std::endl;
if (zstream->avail_out == 0) {
std::cout << "gzip write capacity " << _capacity << std::endl;
_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() {
std::cout << "gzip flush called";
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() {
std::cout << "gzip close called";
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) {
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) {
// printf("return %d\n", length);
return length;
}
} else {
// printf("Error: %d\n", ret);
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

View File

@ -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 "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_ */

View File

@ -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.
*/
#include "commons.h"
#include "NativeTask.h"
#include "Lz4Codec.h"
extern "C" {
extern int LZ4_compress(char* source, char* dest, int isize);
extern int LZ4_uncompress(char* source, char* dest, int osize);
/*
LZ4_compress() :
return : the number of bytes in compressed buffer dest
note : destination buffer must be already allocated.
To avoid any problem, size it to handle worst cases situations (input data not compressible)
Worst case size is : "inputsize + 0.4%", with "0.4%" being at least 8 bytes.
LZ4_uncompress() :
osize : is the output size, therefore the original size
return : the number of bytes read in the source buffer
If the source stream is malformed, the function will stop decoding and return a negative result, indicating the byte position of the faulty instruction
This version never writes beyond dest + osize, and is therefore protected against malicious data packets
note 2 : destination buffer must be already allocated
*/
}
namespace NativeTask {
static int32_t LZ4_MaxCompressedSize(int32_t orig) {
return std::max((int32_t)(orig * 1.005), orig + 8);
}
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

View File

@ -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 "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_ */

View File

@ -0,0 +1,93 @@
/**
* 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 "snappy-c.h"
#include "commons.h"
#include "NativeTask.h"
#include "SnappyCodec.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

View File

@ -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 "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_ */

View File

@ -0,0 +1,138 @@
/*
* Copyright 2011 Martin Gieseking <martin.gieseking@uos.de>.
*
* 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.
*
* Plain C interface (a wrapper around the C++ implementation).
*/
#ifndef UTIL_SNAPPY_OPENSOURCE_SNAPPY_C_H_
#define UTIL_SNAPPY_OPENSOURCE_SNAPPY_C_H_
#ifdef __cplusplus
extern "C" {
#endif
#include <stddef.h>
/*
* Return values; see the documentation for each function to know
* what each can return.
*/
typedef enum {
SNAPPY_OK = 0,
SNAPPY_INVALID_INPUT = 1,
SNAPPY_BUFFER_TOO_SMALL = 2
} snappy_status;
/*
* Takes the data stored in "input[0..input_length-1]" and stores
* it in the array pointed to by "compressed".
*
* <compressed_length> signals the space available in "compressed".
* If it is not at least equal to "snappy_max_compressed_length(input_length)",
* SNAPPY_BUFFER_TOO_SMALL is returned. After successful compression,
* <compressed_length> contains the true length of the compressed output,
* and SNAPPY_OK is returned.
*
* Example:
* size_t output_length = snappy_max_compressed_length(input_length);
* char* output = (char*)malloc(output_length);
* if (snappy_compress(input, input_length, output, &output_length)
* == SNAPPY_OK) {
* ... Process(output, output_length) ...
* }
* free(output);
*/
snappy_status snappy_compress(const char* input,
size_t input_length,
char* compressed,
size_t* compressed_length);
/*
* Given data in "compressed[0..compressed_length-1]" generated by
* calling the snappy_compress routine, this routine stores
* the uncompressed data to
* uncompressed[0..uncompressed_length-1].
* Returns failure (a value not equal to SNAPPY_OK) if the message
* is corrupted and could not be decrypted.
*
* <uncompressed_length> signals the space available in "uncompressed".
* If it is not at least equal to the value returned by
* snappy_uncompressed_length for this stream, SNAPPY_BUFFER_TOO_SMALL
* is returned. After successful decompression, <uncompressed_length>
* contains the true length of the decompressed output.
*
* Example:
* size_t output_length;
* if (snappy_uncompressed_length(input, input_length, &output_length)
* != SNAPPY_OK) {
* ... fail ...
* }
* char* output = (char*)malloc(output_length);
* if (snappy_uncompress(input, input_length, output, &output_length)
* == SNAPPY_OK) {
* ... Process(output, output_length) ...
* }
* free(output);
*/
snappy_status snappy_uncompress(const char* compressed,
size_t compressed_length,
char* uncompressed,
size_t* uncompressed_length);
/*
* Returns the maximal size of the compressed representation of
* input data that is "source_length" bytes in length.
*/
size_t snappy_max_compressed_length(size_t source_length);
/*
* REQUIRES: "compressed[]" was produced by snappy_compress()
* Returns SNAPPY_OK and stores the length of the uncompressed data in
* *result normally. Returns SNAPPY_INVALID_INPUT on parsing error.
* This operation takes O(1) time.
*/
snappy_status snappy_uncompressed_length(const char* compressed,
size_t compressed_length,
size_t* result);
/*
* Check if the contents of "compressed[]" can be uncompressed successfully.
* Does not return the uncompressed data; if so, returns SNAPPY_OK,
* or if not, returns SNAPPY_INVALID_INPUT.
* Takes time proportional to compressed_length, but is usually at least a
* factor of four faster than actual decompression.
*/
snappy_status snappy_validate_compressed_buffer(const char* compressed,
size_t compressed_length);
#ifdef __cplusplus
} // extern "C"
#endif
#endif /* UTIL_SNAPPY_OPENSOURCE_SNAPPY_C_H_ */

View File

@ -0,0 +1,163 @@
// Copyright 2005 and onwards Google Inc.
//
// 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.
//
// A light-weight compression algorithm. It is designed for speed of
// compression and decompression, rather than for the utmost in space
// savings.
//
// For getting better compression ratios when you are compressing data
// with long repeated sequences or compressing data that is similar to
// other data, while still compressing fast, you might look at first
// using BMDiff and then compressing the output of BMDiff with
// Snappy.
#ifndef UTIL_SNAPPY_SNAPPY_H__
#define UTIL_SNAPPY_SNAPPY_H__
#include <stddef.h>
#include <string>
#include "snappy-stubs-public.h"
namespace snappy {
class Source;
class Sink;
// ------------------------------------------------------------------------
// Generic compression/decompression routines.
// ------------------------------------------------------------------------
// Compress the bytes read from "*source" and append to "*sink". Return the
// number of bytes written.
size_t Compress(Source* source, Sink* sink);
// Find the uncompressed length of the given stream, as given by the header.
// Note that the true length could deviate from this; the stream could e.g.
// be truncated.
//
// Also note that this leaves "*source" in a state that is unsuitable for
// further operations, such as RawUncompress(). You will need to rewind
// or recreate the source yourself before attempting any further calls.
bool GetUncompressedLength(Source* source, uint32* result);
// ------------------------------------------------------------------------
// Higher-level string based routines (should be sufficient for most users)
// ------------------------------------------------------------------------
// Sets "*output" to the compressed version of "input[0,input_length-1]".
// Original contents of *output are lost.
//
// REQUIRES: "input[]" is not an alias of "*output".
size_t Compress(const char* input, size_t input_length, string* output);
// Decompresses "compressed[0,compressed_length-1]" to "*uncompressed".
// Original contents of "*uncompressed" are lost.
//
// REQUIRES: "compressed[]" is not an alias of "*uncompressed".
//
// returns false if the message is corrupted and could not be decompressed
bool Uncompress(const char* compressed, size_t compressed_length,
string* uncompressed);
// ------------------------------------------------------------------------
// Lower-level character array based routines. May be useful for
// efficiency reasons in certain circumstances.
// ------------------------------------------------------------------------
// REQUIRES: "compressed" must point to an area of memory that is at
// least "MaxCompressedLength(input_length)" bytes in length.
//
// Takes the data stored in "input[0..input_length]" and stores
// it in the array pointed to by "compressed".
//
// "*compressed_length" is set to the length of the compressed output.
//
// Example:
// char* output = new char[snappy::MaxCompressedLength(input_length)];
// size_t output_length;
// RawCompress(input, input_length, output, &output_length);
// ... Process(output, output_length) ...
// delete [] output;
void RawCompress(const char* input,
size_t input_length,
char* compressed,
size_t* compressed_length);
// Given data in "compressed[0..compressed_length-1]" generated by
// calling the Snappy::Compress routine, this routine
// stores the uncompressed data to
// uncompressed[0..GetUncompressedLength(compressed)-1]
// returns false if the message is corrupted and could not be decrypted
bool RawUncompress(const char* compressed, size_t compressed_length,
char* uncompressed);
// Given data from the byte source 'compressed' generated by calling
// the Snappy::Compress routine, this routine stores the uncompressed
// data to
// uncompressed[0..GetUncompressedLength(compressed,compressed_length)-1]
// returns false if the message is corrupted and could not be decrypted
bool RawUncompress(Source* compressed, char* uncompressed);
// Returns the maximal size of the compressed representation of
// input data that is "source_bytes" bytes in length;
size_t MaxCompressedLength(size_t source_bytes);
// REQUIRES: "compressed[]" was produced by RawCompress() or Compress()
// Returns true and stores the length of the uncompressed data in
// *result normally. Returns false on parsing error.
// This operation takes O(1) time.
bool GetUncompressedLength(const char* compressed, size_t compressed_length,
size_t* result);
// Returns true iff the contents of "compressed[]" can be uncompressed
// successfully. Does not return the uncompressed data. Takes
// time proportional to compressed_length, but is usually at least
// a factor of four faster than actual decompression.
bool IsValidCompressedBuffer(const char* compressed,
size_t compressed_length);
// The size of a compression block. Note that many parts of the compression
// code assumes that kBlockSize <= 65536; in particular, the hash table
// can only store 16-bit offsets, and EmitCopy() also assumes the offset
// is 65535 bytes or less. Note also that if you change this, it will
// affect the framing format (see framing_format.txt).
//
// Note that there might be older data around that is compressed with larger
// block sizes, so the decompression code should not rely on the
// non-existence of long backreferences.
static const int kBlockLog = 16;
static const size_t kBlockSize = 1 << kBlockLog;
static const int kMaxHashTableBits = 14;
static const size_t kMaxHashTableSize = 1 << kMaxHashTableBits;
} // end namespace snappy
#endif // UTIL_SNAPPY_SNAPPY_H__

View 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.
*/
#include "commons.h"
#include "util/StringUtil.h"
#include "MCollectorOutputHandler.h"
#include "NativeObjectFactory.h"
#include "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

View File

@ -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_ */

View File

@ -0,0 +1,354 @@
/*
* 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 "commons.h"
#include "jni_md.h"
#include "jniutils.h"
#include "BatchHandler.h"
#include "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");
}

View File

@ -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) {
if (length > _out.remain()) {
flushOutput();
}
uint32_t remain = _out.remain();
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_ */

View File

@ -0,0 +1,256 @@
/**
* 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), _config(NULL), _kvCached(false),
_kType(UnknownType), _vType(UnknownType), _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) {
uint32_t length = 0;
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();
const char * end = buff + length;
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");
}
int 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 */

View File

@ -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 "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_ */

View File

@ -0,0 +1,97 @@
/*
* 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 "commons.h"
#include "util/StringUtil.h"
#include "MCollectorOutputHandler.h"
#include "NativeObjectFactory.h"
#include "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

View File

@ -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_ */

Some files were not shown because too many files have changed in this diff Show More