libraryLoaded = new AtomicReference<>(LibraryState.NOT_LOADED);
+
+    static {
+        NativeLibrary.loadLibrary();
+    }
+
+    public static void loadLibrary() {
+        if (libraryLoaded.get() == LibraryState.LOADED) {
+            return;
+        }
+
+        if (libraryLoaded.compareAndSet(LibraryState.NOT_LOADED, LibraryState.LOADING)) {
+            try {
+                doLoadLibrary();
+            } catch (IOException e) {
+                libraryLoaded.set(LibraryState.NOT_LOADED);
+                throw new UncheckedIOException("Unable to load the Databend shared library", e);
+            }
+            libraryLoaded.set(LibraryState.LOADED);
+            return;
+        }
+
+        while (libraryLoaded.get() == LibraryState.LOADING) {
+            try {
+                Thread.sleep(10);
+            } catch (InterruptedException ignore) {
+            }
+        }
+    }
+
+    private static void doLoadLibrary() throws IOException {
+        try {
+            // try dynamic library - the search path can be configured via "-Djava.library.path"
+            System.loadLibrary("bendsql_java");
+            return;
+        } catch (UnsatisfiedLinkError ignore) {
+            // ignore - try from classpath
+        }
+
+        doLoadBundledLibrary();
+    }
+
+    private static void doLoadBundledLibrary() throws IOException {
+        final String libraryPath = bundledLibraryPath();
+        try (final InputStream is = NativeObject.class.getResourceAsStream(libraryPath)) {
+            if (is == null) {
+                throw new IOException("cannot find " + libraryPath);
+            }
+            final int dot = libraryPath.indexOf('.');
+            final File tmpFile = File.createTempFile(libraryPath.substring(0, dot), libraryPath.substring(dot));
+            tmpFile.deleteOnExit();
+            Files.copy(is, tmpFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+            System.load(tmpFile.getAbsolutePath());
+        }
+    }
+
+    private static String bundledLibraryPath() {
+        final String classifier = Environment.getClassifier();
+        final String libraryName = System.mapLibraryName("bendsql_java");
+        return "/native/" + classifier + "/" + libraryName;
+    }
+}
diff --git a/bindings/java/src/main/java/com/databend/bendsql/jni_utils/NativeObject.java b/bindings/java/src/main/java/com/databend/bendsql/jni_utils/NativeObject.java
new file mode 100644
index 000000000..c1a5bd9d0
--- /dev/null
+++ b/bindings/java/src/main/java/com/databend/bendsql/jni_utils/NativeObject.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.databend.bendsql.jni_utils;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * NativeObject is the base-class of all BendSQL classes that have
+ * a pointer to a native object.
+ *
+ * 
+ * NativeObject has the {@link NativeObject#close()} method, which frees its associated
+ * native object.
+ *
+ * 
+ * This function should be called manually, or even better, called implicitly using a
+ * try-with-resources
+ * statement, when you are finished with the object. It is no longer called automatically
+ * during the regular Java GC process via {@link NativeObject#finalize()}.
+ *
+ * 
+ * Explanatory note
+ *
+ * 
+ * When or if the Garbage Collector calls {@link Object#finalize()}
+ * depends on the JVM implementation and system conditions, which the programmer
+ * cannot control. In addition, the GC cannot see through the native reference
+ * long member variable (which is the pointer value to the native object),
+ * and cannot know what other resources depend on it.
+ *
+ * 
+ * Finalization is deprecated and subject to removal in a future release.
+ * The use of finalization can lead to problems with security, performance,
+ * and reliability. See JEP 421
+ * for discussion and alternatives.
+ */
+public abstract class NativeObject implements AutoCloseable {
+    static {
+        NativeLibrary.loadLibrary();
+    }
+
+    private final AtomicBoolean disposed = new AtomicBoolean(false);
+
+/**
+     * An immutable reference to the value of the underneath pointer pointing
+     * to some underlying native Databend object.
+     */
+    protected final long nativeHandle;
+    protected final long executorHandle;
+
+    protected NativeObject(long nativeHandle, long executorHandle) {
+        this.nativeHandle = nativeHandle;
+        this.executorHandle = executorHandle;
+    }
+
+    @Override
+    public void close() {
+        if (disposed.compareAndSet(false, true)) {
+            disposeInternal(nativeHandle, executorHandle);
+        }
+    }
+
+    /**
+     * Check if the object has been disposed. Useful for defensive programming.
+     *
+     * @return if the object has been disposed.
+     */
+    public boolean isDisposed() {
+        return disposed.get();
+    }
+
+    /**
+     * Deletes underlying native object pointer.
+     *
+     * @param handle to the native object pointer
+     */
+    protected abstract void disposeInternal(long handle, long executorHandle);
+
+    public long getNativeHandle() {
+        return nativeHandle;
+    }
+}
diff --git a/bindings/java/src/main/resources/META-INF/services/java.sql.Driver b/bindings/java/src/main/resources/META-INF/services/java.sql.Driver
new file mode 100644
index 000000000..f2b3502f9
--- /dev/null
+++ b/bindings/java/src/main/resources/META-INF/services/java.sql.Driver
@@ -0,0 +1 @@
+com.databend.bendsql.DatabendDriver
\ No newline at end of file
diff --git a/bindings/java/src/main/resources/bindings.properties b/bindings/java/src/main/resources/bindings.properties
new file mode 100644
index 000000000..ad06a8d04
--- /dev/null
+++ b/bindings/java/src/main/resources/bindings.properties
@@ -0,0 +1,15 @@
+# Copyright 2021 Datafuse Labs
+#
+# 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.
+
+project.version=${project.version}
diff --git a/bindings/java/src/row_batch_iterator.rs b/bindings/java/src/row_batch_iterator.rs
new file mode 100644
index 000000000..eb6bdfa38
--- /dev/null
+++ b/bindings/java/src/row_batch_iterator.rs
@@ -0,0 +1,88 @@
+// Copyright 2021 Datafuse Labs
+//
+// 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.
+
+use crate::jni_utils::executor::executor_or_default;
+use crate::jni_utils::executor::Executor;
+use crate::Result;
+use jni::objects::JClass;
+
+use databend_driver::rest_api::RowBatch;
+use jni::sys::jstring;
+use jni::sys::{jlong, jobject};
+
+use jni::JNIEnv;
+
+#[no_mangle]
+pub extern "system" fn Java_com_databend_bendsql_NativeRowBatchIterator_fetchNextRowBatch(
+    mut env: JNIEnv,
+    _class: JClass,
+    it: *mut RowBatch,
+    executor: *const Executor,
+) -> jstring {
+    fetch_next_row_batch(&mut env, it, executor).unwrap_or_else(|e| {
+        e.throw(&mut env);
+        std::ptr::null_mut()
+    })
+}
+
+fn fetch_next_row_batch(
+    env: &mut JNIEnv,
+    it: *mut RowBatch,
+    executor: *const Executor,
+) -> Result {
+    let batch = unsafe { &mut *it };
+
+    let data = executor_or_default(env, executor)?
+        .block_on(async move { batch.fetch_next_page().await })?;
+
+    if !data.is_empty() {
+        let json = serde_json::to_string(&data).unwrap();
+        let jstring = env.new_string(json)?;
+        Ok(jstring.into_raw())
+    } else {
+        Ok(std::ptr::null_mut())
+    }
+}
+
+#[no_mangle]
+pub extern "system" fn Java_com_databend_bendsql_NativeRowBatchIterator_getSchema(
+    mut env: JNIEnv,
+    _class: JClass,
+    it: *mut RowBatch,
+) -> jstring {
+    get_schema(&mut env, it).unwrap_or_else(|e| {
+        e.throw(&mut env);
+        std::ptr::null_mut()
+    })
+}
+
+fn get_schema(env: &mut JNIEnv, it: *mut RowBatch) -> Result {
+    let batch = unsafe { &mut *it };
+    let schema = batch.schema();
+    let json = serde_json::to_string(&schema).unwrap();
+    let jstring = env.new_string(json)?;
+    Ok(jstring.into_raw())
+}
+
+#[no_mangle]
+pub extern "system" fn Java_com_databend_bendsql_NativeRowBatchIterator_disposeInternal(
+    _env: JNIEnv,
+    _class: JClass,
+    handle: jlong,
+    _executor: *const Executor,
+) {
+    if handle != 0 {
+        let _ = unsafe { Box::from_raw(handle as *mut RowBatch) };
+    }
+}
diff --git a/bindings/java/src/test/java/com/databend/bendsql/DatabendConnectionTest.java b/bindings/java/src/test/java/com/databend/bendsql/DatabendConnectionTest.java
new file mode 100644
index 000000000..b24a6b291
--- /dev/null
+++ b/bindings/java/src/test/java/com/databend/bendsql/DatabendConnectionTest.java
@@ -0,0 +1,55 @@
+package com.databend.bendsql;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Properties;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class DatabendConnectionTest {
+    
+    private DatabendConnection connection;
+    private static final String TEST_DSN = "databend://root:@localhost:8000/default?sslmode=disable";
+    
+    @BeforeEach
+    void setUp() throws Exception {
+        Properties props = new Properties();
+        connection = new DatabendConnection(TEST_DSN, props);
+    }
+    
+    @Test
+    void testCreateStatement() throws Exception {
+        Statement stmt = connection.createStatement();
+        assertNotNull(stmt);
+        assertTrue(stmt instanceof DatabendStatement);
+    }
+    
+    @Test
+    void testSimpleQuery() throws Exception {
+        Statement stmt = connection.createStatement();
+        
+        ResultSet rs = stmt.executeQuery("SELECT 1");
+        
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertFalse(rs.next());
+    }
+    
+    @Test
+    void testTableQuery() throws Exception {
+        Statement stmt = connection.createStatement();
+        
+        stmt.execute("CREATE OR REPLACE TABLE test_table (id INT, name VARCHAR)");
+        stmt.execute("INSERT INTO test_table VALUES (1, 'test')");
+        ResultSet rs = stmt.executeQuery("SELECT * FROM test_table");
+        
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt("id"));
+        assertEquals("test", rs.getString("name"));
+        assertFalse(rs.next());
+        
+        stmt.execute("DROP TABLE IF EXISTS test_table");
+    }
+} 
\ No newline at end of file
diff --git a/bindings/java/src/test/java/com/databend/bendsql/DatabendDriverTest.java b/bindings/java/src/test/java/com/databend/bendsql/DatabendDriverTest.java
new file mode 100644
index 000000000..df7f1496f
--- /dev/null
+++ b/bindings/java/src/test/java/com/databend/bendsql/DatabendDriverTest.java
@@ -0,0 +1,64 @@
+package com.databend.bendsql;
+
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.*;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.List;
+import static com.databend.bendsql.utils.ResultSetTestUtils.assertResultSet;
+
+public class DatabendDriverTest {
+    private static final String TEST_DSN = "jdbc:databend://root:@localhost:8000/default?sslmode=disable";
+
+    @Test
+    public void testSimpleSelect() throws Exception {
+        try (Connection conn = DriverManager.getConnection(TEST_DSN, null, null);
+                Statement stmt = conn.createStatement();
+                ResultSet rs = stmt.executeQuery("SELECT 1, 'hello'")) {
+
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("hello", rs.getString(2));
+
+            assertFalse(rs.next());
+        }
+    }
+
+    @Test
+    public void testBatchInsert() throws Exception {
+        try (Connection conn = DriverManager.getConnection(TEST_DSN, null, null);) {
+            try(Statement stmt = conn.createStatement();) {
+                stmt.execute("create or replace table test_prepare_statement (a int, b string)");
+            }
+
+            try(PreparedStatement ps = conn.prepareStatement("insert into test_prepare_statement values");) {
+                ps.setInt(1, 1);
+                ps.setString(2, "a");
+                ps.addBatch();
+                ps.setInt(1, 2);
+                ps.setString(2, "b");
+                ps.addBatch(); 
+                int[] ans = ps.executeBatch();
+                assertEquals(ans.length, 2);
+                //assertEquals(ans[0], 1);
+                //assertEquals(ans[1], 1);
+                Statement statement = conn.createStatement();
+
+                boolean hasResultSet = statement.execute("SELECT * from test_prepare_statement");
+                assertTrue(hasResultSet);
+                try(ResultSet rs = statement.getResultSet();) {
+                    List