You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wh...@apache.org on 2016/01/05 20:52:29 UTC

[29/50] [abbrv] hadoop git commit: [partial-ns] Implement JNI bindings for HDFSDB.

[partial-ns] Implement JNI bindings for HDFSDB.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8a8b70db
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8a8b70db
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8a8b70db

Branch: refs/heads/feature-HDFS-8286
Commit: 8a8b70db882607a991f896e9e651dc9328e7b98a
Parents: cb5ba73
Author: Haohui Mai <wh...@apache.org>
Authored: Fri Nov 7 18:06:22 2014 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Jun 12 13:57:00 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   5 +
 hadoop-hdfs-project/hadoop-hdfsdb/pom.xml       |  25 ++
 .../java/org/apache/hadoop/hdfs/hdfsdb/DB.java  |  70 +++++
 .../org/apache/hadoop/hdfs/hdfsdb/Iterator.java |  75 ++++++
 .../apache/hadoop/hdfs/hdfsdb/NativeObject.java |  35 +++
 .../org/apache/hadoop/hdfs/hdfsdb/Options.java  |  73 ++++++
 .../apache/hadoop/hdfs/hdfsdb/ReadOptions.java  |  35 +++
 .../apache/hadoop/hdfs/hdfsdb/WriteBatch.java   |  50 ++++
 .../apache/hadoop/hdfs/hdfsdb/WriteOptions.java |  35 +++
 .../src/main/native/CMakeLists.txt              |   1 +
 .../src/main/native/jni/CMakeLists.txt          |  13 +
 .../src/main/native/jni/bindings.cc             | 254 +++++++++++++++++++
 hadoop-project/pom.xml                          |   5 +
 13 files changed, 676 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a8b70db/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index 34338d1..e36b409 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -56,6 +56,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfsdb</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <scope>test</scope>
       <type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a8b70db/hadoop-hdfs-project/hadoop-hdfsdb/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfsdb/pom.xml b/hadoop-hdfs-project/hadoop-hdfsdb/pom.xml
index 1eef71e..34a5b3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfsdb/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfsdb/pom.xml
@@ -43,6 +43,31 @@
   <build>
     <plugins>
       <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>
+                <javahClassName>org.apache.hadoop.hdfs.hdfsdb.NativeObject</javahClassName>
+                <javahClassName>org.apache.hadoop.hdfs.hdfsdb.DB</javahClassName>
+                <javahClassName>org.apache.hadoop.hdfs.hdfsdb.Iterator</javahClassName>
+                <javahClassName>org.apache.hadoop.hdfs.hdfsdb.Options</javahClassName>
+                <javahClassName>org.apache.hadoop.hdfs.hdfsdb.ReadOptions</javahClassName>
+                <javahClassName>org.apache.hadoop.hdfs.hdfsdb.WriteOptions</javahClassName>
+                <javahClassName>org.apache.hadoop.hdfs.hdfsdb.WriteBatch</javahClassName>
+              </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>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a8b70db/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/DB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/DB.java b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/DB.java
new file mode 100644
index 0000000..0355dcc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/DB.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.hdfsdb;
+
+import java.io.IOException;
+
+public class DB extends NativeObject {
+  public static DB open(Options options, String path) throws IOException {
+    return new DB(open(options.nativeHandle(), path));
+  }
+
+  @Override
+  public void close() {
+    if (this.nativeHandle != 0) {
+      close(nativeHandle);
+      nativeHandle = 0;
+    }
+  }
+
+  public byte[] get(ReadOptions options, byte[] key) throws IOException {
+    return get(nativeHandle, options.nativeHandle(), key);
+  }
+
+  public void write(WriteOptions options, WriteBatch batch) throws IOException {
+    write(nativeHandle, options.nativeHandle(), batch.nativeHandle());
+  }
+
+  public void put(WriteOptions options, byte[] key, byte[] value) {
+    put(nativeHandle, options.nativeHandle(), key, value);
+  }
+
+  public void delete(WriteOptions options, byte[] key) {
+    delete(nativeHandle, options.nativeHandle(), key);
+  }
+
+  public Iterator iterator(ReadOptions options) {
+    return new Iterator(newIterator(nativeHandle, options.nativeHandle()));
+  }
+
+  private DB(long handle) {
+    super(handle);
+  }
+
+  private static native long open(long options, String path) throws IOException;
+  private static native void close(long handle);
+  private static native byte[] get(long handle, long options,
+                                   byte[] key) throws IOException;
+  private static native void write(long handle, long options,
+                                   long batch) throws IOException;
+  private static native void put(long handle, long options,
+                                 byte[] key, byte[] value);
+  private static native void delete(long handle, long options,
+                                    byte[] key);
+  private static native long newIterator(long handle, long readOptions);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a8b70db/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/Iterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/Iterator.java b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/Iterator.java
new file mode 100644
index 0000000..312bd02
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/Iterator.java
@@ -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.hdfs.hdfsdb;
+
+import java.util.AbstractMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+public class Iterator extends NativeObject implements java.util.Iterator<Map
+        .Entry<byte[], byte[]>> {
+
+  Iterator(long nativeHandle) {
+    super(nativeHandle);
+  }
+
+  public void seek(byte[] key) {
+    seek(nativeHandle, key);
+  }
+
+  @Override
+  public boolean hasNext() {
+    return valid(nativeHandle);
+  }
+
+  @Override
+  public Map.Entry<byte[], byte[]> next() {
+    Map.Entry<byte[], byte[]> res = peekNext();
+    next(nativeHandle);
+    return res;
+  }
+
+  public Map.Entry<byte[], byte[]> peekNext() {
+    if (!hasNext()) {
+      throw new NoSuchElementException();
+    }
+
+    return new AbstractMap.SimpleImmutableEntry<byte[], byte[]>(key
+            (nativeHandle), value(nativeHandle));
+  }
+
+  @Override
+  public void remove() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void close() {
+    if (nativeHandle != 0) {
+      destruct(nativeHandle);
+      nativeHandle = 0;
+    }
+  }
+
+  private static native void destruct(long handle);
+  private static native void seek(long handle, byte[] key);
+  private static native void next(long handle);
+  private static native boolean valid(long handle);
+  private static native byte[] key(long handle);
+  private static native byte[] value(long handle);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a8b70db/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/NativeObject.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/NativeObject.java b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/NativeObject.java
new file mode 100644
index 0000000..0290e84
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/NativeObject.java
@@ -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.hdfs.hdfsdb;
+
+abstract class NativeObject implements AutoCloseable {
+  static {
+    System.loadLibrary("hdfs-jni");
+  }
+  protected long nativeHandle;
+  protected long nativeHandle() { return nativeHandle; }
+  protected NativeObject(long nativeHandle) {
+    this.nativeHandle = nativeHandle;
+  }
+
+  @Override
+  protected void finalize() throws Throwable {
+    close();
+    super.finalize();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a8b70db/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/Options.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/Options.java b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/Options.java
new file mode 100644
index 0000000..a12da61
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/Options.java
@@ -0,0 +1,73 @@
+/**
+ * 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.hdfs.hdfsdb;
+
+public class Options extends NativeObject {
+  public enum CompressionType {
+    NO_COMPRESSION(0),
+    SNAPPY(1);
+
+    final int value;
+    CompressionType(int value) {
+      this.value = value;
+    }
+  }
+  public Options() {
+    super(construct());
+  }
+
+  public Options createIfMissing(boolean value) {
+    createIfMissing(nativeHandle, value);
+    return this;
+  }
+
+  public Options compressionType(int type) {
+    CompressionType ctype = CompressionType.values()[type];
+    return compressionType(ctype);
+  }
+
+  public Options compressionType(CompressionType type) {
+    compressionType(nativeHandle, type.value);
+    return this;
+  }
+
+  public Options writeBufferSize(int value) {
+    writeBufferSize(nativeHandle, value);
+    return this;
+  }
+
+  public Options blockSize(int value) {
+    blockSize(nativeHandle, value);
+    return this;
+  }
+
+  @Override
+  public void close() {
+    if (nativeHandle != 0) {
+      destruct(nativeHandle);
+      nativeHandle = 0;
+    }
+  }
+
+  private static native long construct();
+  private static native void destruct(long handle);
+  private static native void createIfMissing(long handle, boolean value);
+  private static native void compressionType(long handle, int value);
+  private static native void writeBufferSize(long handle, int value);
+  private static native void blockSize(long handle, int value);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a8b70db/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/ReadOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/ReadOptions.java b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/ReadOptions.java
new file mode 100644
index 0000000..e97e05f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/ReadOptions.java
@@ -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.hdfs.hdfsdb;
+
+public class ReadOptions extends NativeObject {
+  public ReadOptions() {
+    super(construct());
+  }
+
+  @Override
+  public void close() {
+    if (nativeHandle != 0) {
+      destruct(nativeHandle);
+      nativeHandle = 0;
+    }
+  }
+
+  private static native long construct();
+  private static native void destruct(long handle);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a8b70db/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/WriteBatch.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/WriteBatch.java b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/WriteBatch.java
new file mode 100644
index 0000000..40466a0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/WriteBatch.java
@@ -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.hdfs.hdfsdb;
+
+public class WriteBatch extends NativeObject {
+  public WriteBatch() {
+    super(construct());
+  }
+
+  public void put(byte[] key, byte[] value) {
+    put(nativeHandle, key, value);
+  }
+
+  public void delete(byte[] key) {
+    delete(nativeHandle, key);
+  }
+
+  public void clear() {
+    clear(nativeHandle);
+  }
+
+  @Override
+  public void close() {
+    if (nativeHandle != 0) {
+      destruct(nativeHandle);
+      nativeHandle = 0;
+    }
+  }
+
+  private static native long construct();
+  private static native void put(long handle, byte[] key, byte[] value);
+  private static native void delete(long handle, byte[] key);
+  private static native void clear(long handle);
+  private static native void destruct(long handle);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a8b70db/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/WriteOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/WriteOptions.java b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/WriteOptions.java
new file mode 100644
index 0000000..cd6d2b7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/java/org/apache/hadoop/hdfs/hdfsdb/WriteOptions.java
@@ -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.hdfs.hdfsdb;
+
+public class WriteOptions extends NativeObject {
+  public WriteOptions() {
+    super(construct());
+  }
+
+  @Override
+  public void close() {
+    if (nativeHandle != 0) {
+      destruct(nativeHandle);
+      nativeHandle = 0;
+    }
+  }
+
+  private static native long construct();
+  private static native void destruct(long handle);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a8b70db/hadoop-hdfs-project/hadoop-hdfsdb/src/main/native/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfsdb/src/main/native/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/native/CMakeLists.txt
index d21b897..a1e78fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfsdb/src/main/native/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/native/CMakeLists.txt
@@ -9,3 +9,4 @@ set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11")
 endif()
 
 add_subdirectory(hdfsdb)
+add_subdirectory(jni)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a8b70db/hadoop-hdfs-project/hadoop-hdfsdb/src/main/native/jni/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfsdb/src/main/native/jni/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/native/jni/CMakeLists.txt
new file mode 100644
index 0000000..ea0d7a5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/native/jni/CMakeLists.txt
@@ -0,0 +1,13 @@
+find_package(JNI REQUIRED)
+
+cmake_policy(SET CMP0042 NEW)
+
+include_directories(${CMAKE_BINARY_DIR}/javah ${CMAKE_SOURCE_DIR}/hdfsdb/include)
+include_directories(${JAVA_INCLUDE_PATH} ${JAVA_INCLUDE_PATH2})
+
+add_library(hdfsdb-jni SHARED bindings.cc)
+target_link_libraries(hdfsdb-jni hdfsdb)
+
+set_target_properties(hdfsdb-jni PROPERTIES RUNTIME_OUTPUT_DIRECTORY ${CMAKE_BINARY_DIR}/target/usr/local/lib)
+set_target_properties(hdfsdb-jni PROPERTIES ARCHIVE_OUTPUT_DIRECTORY ${CMAKE_BINARY_DIR}/target/usr/local/lib)
+set_target_properties(hdfsdb-jni PROPERTIES LIBRARY_OUTPUT_DIRECTORY ${CMAKE_BINARY_DIR}/target/usr/local/lib)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a8b70db/hadoop-hdfs-project/hadoop-hdfsdb/src/main/native/jni/bindings.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfsdb/src/main/native/jni/bindings.cc b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/native/jni/bindings.cc
new file mode 100644
index 0000000..6682159
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfsdb/src/main/native/jni/bindings.cc
@@ -0,0 +1,254 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <jni.h>
+
+#undef JNIEXPORT
+#if _WIN32
+#define JNIEXPORT __declspec(dllexport)
+#else
+#define JNIEXPORT __attribute__((visibility("default")))
+#endif
+
+#include "org_apache_hadoop_hdfs_hdfsdb_DB.h"
+#include "org_apache_hadoop_hdfs_hdfsdb_Iterator.h"
+#include "org_apache_hadoop_hdfs_hdfsdb_Options.h"
+#include "org_apache_hadoop_hdfs_hdfsdb_NativeObject.h"
+#include "org_apache_hadoop_hdfs_hdfsdb_ReadOptions.h"
+#include "org_apache_hadoop_hdfs_hdfsdb_WriteBatch.h"
+#include "org_apache_hadoop_hdfs_hdfsdb_WriteOptions.h"
+
+#include <leveldb/db.h>
+#include <leveldb/options.h>
+#include <leveldb/write_batch.h>
+#include <leveldb/cache.h>
+
+static inline uintptr_t uintptr(void *ptr) {
+  return reinterpret_cast<uintptr_t>(ptr);
+}
+
+static inline jbyteArray ToJByteArray(JNIEnv *env, const leveldb::Slice &slice) {
+  jbyteArray res = env->NewByteArray(slice.size());
+  if (!res) {
+    env->ThrowNew(env->FindClass("java/lang/OutOfMemoryError"), "");
+  } else {
+    env->SetByteArrayRegion(res, 0, slice.size(), reinterpret_cast<const jbyte*>(slice.data()));
+  }
+  return res;
+}
+
+struct GetByteArrayElements {
+  static jbyte *Get(JNIEnv *env, jbyteArray array) {
+    return env->GetByteArrayElements(array, NULL);
+  }
+  static void Release(JNIEnv *env, jbyteArray array, jbyte *data) {
+    env->ReleaseByteArrayElements(array, data, JNI_ABORT);
+  }
+};
+
+struct GetByteArrayCritical {
+  static jbyte *Get(JNIEnv *env, jbyteArray array) {
+    return reinterpret_cast<jbyte*>(env->GetPrimitiveArrayCritical(array, NULL));
+  }
+  static void Release(JNIEnv *env, jbyteArray array, jbyte *data) {
+    env->ReleasePrimitiveArrayCritical(array, data, JNI_ABORT);
+  }
+};
+
+template <class Trait>
+class JNIByteArrayHolder {
+ public:
+  JNIByteArrayHolder(JNIEnv *env, jbyteArray array)
+      : env_(env)
+      , array_(array)
+  {
+    data_ = Trait::Get(env, array);
+    length_ = env_->GetArrayLength(array);
+  }
+
+  ~JNIByteArrayHolder() { Trait::Release(env_, array_, data_); }
+  const char *data() const { return reinterpret_cast<const char*>(data_); }
+  int length() const { return length_; }
+  const leveldb::Slice slice() const { return leveldb::Slice(data(), length()); }
+
+ private:
+  JNIEnv *env_;
+  jbyteArray array_;
+  jbyte *data_;
+  int length_;
+};
+
+
+jlong JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_DB_open(JNIEnv *env, jclass, jlong joptions, jstring path) {
+  leveldb::DB *db = NULL;
+  const char *path_str = env->GetStringUTFChars(path, 0);
+  leveldb::Options *options = reinterpret_cast<leveldb::Options*>(joptions);
+  leveldb::Status status = leveldb::DB::Open(*options, path_str, &db);
+  env->ReleaseStringUTFChars(path, path_str);
+  if (!status.ok()) {
+    env->ThrowNew(env->FindClass("java/io/IOException"), status.ToString().c_str());
+  }
+  return uintptr(db);
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_DB_close(JNIEnv *, jclass, jlong handle) {
+  leveldb::DB *db = reinterpret_cast<leveldb::DB*>(handle);
+  delete db;
+}
+
+jbyteArray JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_DB_get(JNIEnv *env, jclass, jlong handle, jlong jread_options, jbyteArray jkey) {
+  leveldb::DB *db = reinterpret_cast<leveldb::DB*>(handle);
+  leveldb::ReadOptions *options = reinterpret_cast<leveldb::ReadOptions*>(jread_options);
+  std::string result;
+  leveldb::Status status;
+  {
+    JNIByteArrayHolder<GetByteArrayElements> key(env, jkey);
+    status = db->Get(*options, key.slice(), &result);
+  }
+
+  if (status.IsNotFound()) {
+    return NULL;
+  } else if (!status.ok()) {
+    env->ThrowNew(env->FindClass("java/io/IOException"), status.ToString().c_str());
+    return NULL;
+  }
+
+  return ToJByteArray(env, leveldb::Slice(result));
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_DB_write(JNIEnv *env, jclass, jlong handle, jlong jwrite_options, jlong jbatch) {
+  leveldb::DB *db = reinterpret_cast<leveldb::DB*>(handle);
+  leveldb::WriteOptions *options = reinterpret_cast<leveldb::WriteOptions*>(jwrite_options);
+  leveldb::WriteBatch *batch = reinterpret_cast<leveldb::WriteBatch*>(jbatch);
+  leveldb::Status status = db->Write(*options, batch);
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_DB_delete(JNIEnv *env, jclass, jlong handle, jlong jwrite_options, jbyteArray jkey) {
+  leveldb::DB *db = reinterpret_cast<leveldb::DB*>(handle);
+  leveldb::WriteOptions *options = reinterpret_cast<leveldb::WriteOptions*>(jwrite_options);
+  leveldb::Status status;
+  {
+    JNIByteArrayHolder<GetByteArrayElements> key(env, jkey);
+    status = db->Delete(*options, key.slice());
+  }
+}
+
+jlong JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_DB_newIterator(JNIEnv *, jclass, jlong handle, jlong jread_options) {
+  leveldb::DB *db = reinterpret_cast<leveldb::DB*>(handle);
+  leveldb::ReadOptions *options = reinterpret_cast<leveldb::ReadOptions*>(jread_options);
+  auto res = uintptr(db->NewIterator(*options));
+  return res;
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_Iterator_destruct(JNIEnv *, jclass, jlong handle) {
+  delete reinterpret_cast<leveldb::Iterator*>(handle);
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_Iterator_seek(JNIEnv *env, jclass, jlong handle, jbyteArray jkey) {
+  leveldb::Iterator *it = reinterpret_cast<leveldb::Iterator*>(handle);
+  JNIByteArrayHolder<GetByteArrayElements> key(env, jkey);
+  it->Seek(key.slice());
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_Iterator_next(JNIEnv *, jclass, jlong handle) {
+  leveldb::Iterator *it = reinterpret_cast<leveldb::Iterator*>(handle);
+  it->Next();
+}
+
+jboolean JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_Iterator_valid(JNIEnv *, jclass, jlong handle) {
+  leveldb::Iterator *it = reinterpret_cast<leveldb::Iterator*>(handle);
+  return it->Valid();
+}
+
+jbyteArray JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_Iterator_key(JNIEnv *env, jclass, jlong handle) {
+  leveldb::Iterator *it = reinterpret_cast<leveldb::Iterator*>(handle);
+  return ToJByteArray(env, it->key());
+}
+
+jbyteArray JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_Iterator_value(JNIEnv *env, jclass, jlong handle) {
+  leveldb::Iterator *it = reinterpret_cast<leveldb::Iterator*>(handle);
+  return ToJByteArray(env, it->value());
+}
+
+jlong JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_Options_construct(JNIEnv *, jclass) {
+  return uintptr(new leveldb::Options());
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_Options_destruct(JNIEnv *, jclass, jlong handle) {
+  delete reinterpret_cast<leveldb::Options*>(handle);
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_Options_createIfMissing(JNIEnv *, jclass, jlong handle, jboolean value) {
+  leveldb::Options *options = reinterpret_cast<leveldb::Options*>(handle);
+  options->create_if_missing = value;
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_Options_compressionType(JNIEnv *, jclass, jlong handle, jint value) {
+  leveldb::Options *options = reinterpret_cast<leveldb::Options*>(handle);
+  options->compression = (leveldb::CompressionType)value;
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_Options_writeBufferSize(JNIEnv *, jclass, jlong handle, jint value) {
+  leveldb::Options *options = reinterpret_cast<leveldb::Options*>(handle);
+  options->write_buffer_size = value;
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_Options_blockSize(JNIEnv *, jclass, jlong handle, jint value) {
+  leveldb::Options *options = reinterpret_cast<leveldb::Options*>(handle);
+  options->block_size = value;
+}
+
+jlong JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_ReadOptions_construct(JNIEnv *, jclass) {
+  return uintptr(new leveldb::ReadOptions());
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_ReadOptions_destruct(JNIEnv *, jclass, jlong handle) {
+  delete reinterpret_cast<leveldb::ReadOptions*>(handle);
+}
+
+jlong JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_WriteOptions_construct(JNIEnv *, jclass) {
+  return uintptr(new leveldb::WriteOptions());
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_WriteOptions_destruct(JNIEnv *, jclass, jlong handle) {
+  delete reinterpret_cast<leveldb::WriteOptions*>(handle);
+}
+
+jlong JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_WriteBatch_construct(JNIEnv *, jclass) {
+  return uintptr(new leveldb::WriteBatch());
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_WriteBatch_destruct(JNIEnv *, jclass, jlong handle) {
+  delete reinterpret_cast<leveldb::WriteBatch*>(handle);
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_WriteBatch_put(JNIEnv *env, jclass, jlong handle, jbyteArray jkey, jbyteArray jvalue) {
+  leveldb::WriteBatch *batch = reinterpret_cast<leveldb::WriteBatch*>(handle);
+  JNIByteArrayHolder<GetByteArrayCritical> key(env, jkey), value(env, jvalue);
+  batch->Put(key.slice(), value.slice());
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_WriteBatch_delete(JNIEnv *env, jclass, jlong handle, jbyteArray jkey) {
+  leveldb::WriteBatch *batch = reinterpret_cast<leveldb::WriteBatch*>(handle);
+  JNIByteArrayHolder<GetByteArrayCritical> key(env, jkey);
+  batch->Delete(key.slice());
+}
+
+void JNICALL Java_org_apache_hadoop_hdfs_hdfsdb_WriteBatch_clear(JNIEnv *, jclass, jlong handle) {
+  leveldb::WriteBatch *batch = reinterpret_cast<leveldb::WriteBatch*>(handle);
+  batch->Clear();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a8b70db/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 1d0262f..5c96b2e 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -166,6 +166,11 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-hdfsdb</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-hdfs</artifactId>
         <version>${project.version}</version>
         <type>test-jar</type>