From a1e80a6ddc34194a80fb60511d88e509e86cedb3 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Mon, 13 May 2019 13:54:23 +0800 Subject: [PATCH 01/21] Add arrow-orc setup setup basic development for native orc reader --- cpp/CMakeLists.txt | 6 ++ cpp/cmake_modules/BuildUtils.cmake | 16 ++- cpp/src/arrow/adapters/orc/CMakeLists.txt | 4 + cpp/src/arrow/adapters/orc/jni/CMakeLists.txt | 55 +++++++++++ .../arrow/adapters/orc/jni/jni_wrapper.cpp | 59 +++++++++++ java/README.md | 9 ++ java/adapter/orc/CMakeLists.txt | 38 ++++++++ java/adapter/orc/pom.xml | 54 +++++++++++ .../adapter/orc/OrcReaderJniWrapper.java | 97 +++++++++++++++++++ java/pom.xml | 15 +++ 10 files changed, 348 insertions(+), 5 deletions(-) create mode 100644 cpp/src/arrow/adapters/orc/jni/CMakeLists.txt create mode 100644 cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp create mode 100644 java/adapter/orc/CMakeLists.txt create mode 100644 java/adapter/orc/pom.xml create mode 100644 java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index c905645e170..740948c43d7 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -252,10 +252,16 @@ endif() if(MSVC) # ORC doesn't build on windows set(ARROW_ORC OFF) + set(ARROW_ORC_JNI OFF) # Plasma using glog is not fully tested on windows. set(ARROW_USE_GLOG OFF) endif() +if(ARROW_ORC_JNI) + set(ARROW_ORC ON) + set(ARROW_BUILD_STATIC ON) +endif() + if(ARROW_ORC) set(ARROW_WITH_LZ4 ON) set(ARROW_WITH_SNAPPY ON) diff --git a/cpp/cmake_modules/BuildUtils.cmake b/cpp/cmake_modules/BuildUtils.cmake index 45cff6e9cca..048ef66e299 100644 --- a/cpp/cmake_modules/BuildUtils.cmake +++ b/cpp/cmake_modules/BuildUtils.cmake @@ -139,7 +139,8 @@ function(ADD_ARROW_LIB LIB_NAME) PRIVATE_INCLUDES DEPENDENCIES SHARED_INSTALL_INTERFACE_LIBS - STATIC_INSTALL_INTERFACE_LIBS) + STATIC_INSTALL_INTERFACE_LIBS + OUTPUT_PATH) cmake_parse_arguments(ARG "${options}" "${one_value_args}" @@ -164,6 +165,11 @@ function(ADD_ARROW_LIB LIB_NAME) else() set(BUILD_STATIC ${ARROW_BUILD_STATIC}) endif() + if(ARG_OUTPUT_PATH) + set(OUTPUT_PATH ${ARG_OUTPUT_PATH}) + else() + set(OUTPUT_PATH ${BUILD_OUTPUT_ROOT_DIRECTORY}) + endif() if(WIN32 OR (CMAKE_GENERATOR STREQUAL Xcode)) # We need to compile C++ separately for each library kind (shared and static) @@ -234,11 +240,11 @@ function(ADD_ARROW_LIB LIB_NAME) set_target_properties(${LIB_NAME}_shared PROPERTIES LIBRARY_OUTPUT_DIRECTORY - "${BUILD_OUTPUT_ROOT_DIRECTORY}" + "${OUTPUT_PATH}" RUNTIME_OUTPUT_DIRECTORY - "${BUILD_OUTPUT_ROOT_DIRECTORY}" + "${OUTPUT_PATH}" PDB_OUTPUT_DIRECTORY - "${BUILD_OUTPUT_ROOT_DIRECTORY}" + "${OUTPUT_PATH}" LINK_FLAGS "${ARG_SHARED_LINK_FLAGS}" OUTPUT_NAME @@ -314,7 +320,7 @@ function(ADD_ARROW_LIB LIB_NAME) set_target_properties(${LIB_NAME}_static PROPERTIES LIBRARY_OUTPUT_DIRECTORY - "${BUILD_OUTPUT_ROOT_DIRECTORY}" OUTPUT_NAME + "${OUTPUT_PATH}" OUTPUT_NAME ${LIB_NAME_STATIC}) if(ARG_STATIC_INSTALL_INTERFACE_LIBS) diff --git a/cpp/src/arrow/adapters/orc/CMakeLists.txt b/cpp/src/arrow/adapters/orc/CMakeLists.txt index 8a29b17d602..c17b4091eda 100644 --- a/cpp/src/arrow/adapters/orc/CMakeLists.txt +++ b/cpp/src/arrow/adapters/orc/CMakeLists.txt @@ -49,3 +49,7 @@ add_arrow_test(adapter-test "orc" STATIC_LINK_LIBS ${ORC_STATIC_TEST_LINK_LIBS}) + +if(ARROW_ORC_JNI) + add_subdirectory(jni) +endif() \ No newline at end of file diff --git a/cpp/src/arrow/adapters/orc/jni/CMakeLists.txt b/cpp/src/arrow/adapters/orc/jni/CMakeLists.txt new file mode 100644 index 00000000000..0088b94e93e --- /dev/null +++ b/cpp/src/arrow/adapters/orc/jni/CMakeLists.txt @@ -0,0 +1,55 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# +# arrow_orc_jni +# + +project(arrow_orc_jni) + +cmake_minimum_required(VERSION 3.11) + +find_package(JNI REQUIRED) + +add_custom_target(arrow_orc_jni) + +set(JNI_HEADERS_DIR "${CMAKE_CURRENT_BINARY_DIR}/generated") + +add_subdirectory(../../../../../../java/adapter/orc ./java) + +set(ARROW_BUILD_STATIC OFF) + +ADD_ARROW_LIB(arrow_orc_jni + BUILD_SHARED + SOURCES + jni_wrapper.cpp + OUTPUTS + ARROW_ORC_JNI_LIBRARIES + SHARED_PRIVATE_LINK_LIBS + arrow_static + EXTRA_INCLUDES + ${JNI_HEADERS_DIR} + PRIVATE_INCLUDES + ${JNI_INCLUDE_DIRS} + ${CMAKE_CURRENT_BINARY_DIR} + DEPENDENCIES + arrow_static + arrow_orc_java + OUTPUT_PATH + ${CMAKE_CURRENT_BINARY_DIR}) + +add_dependencies(arrow_orc_jni ${ARROW_ORC_JNI_LIBRARIES}) \ No newline at end of file diff --git a/cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp b/cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp new file mode 100644 index 00000000000..971535a4a2a --- /dev/null +++ b/cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp @@ -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. + +#include +#include + +#include "org_apache_arrow_adapter_orc_OrcReaderJniWrapper.h" +#include "org_apache_arrow_adapter_orc_OrcReaderJniWrapper_StripeReader.h" + +#ifdef __cplusplus +extern "C" { +#endif + +JNIEXPORT jboolean JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_open + (JNIEnv *env, jobject this_obj, jstring file_path) { + auto in_file = new std::shared_ptr(); + const char *str= env->GetStringUTFChars(file_path, nullptr); + std::string path(str); + env->ReleaseStringUTFChars(file_path, str); + + arrow::Status ret; + if (path.find("hdfs://") == 0) { + return false; + } else { + ret = arrow::io::ReadableFile::Open(path, in_file); + } + + if(ret.ok()) { + + jclass this_class = env->GetObjectClass(this_obj); + jfieldID fidNativeManager = env->GetFieldID(this_class, "nativeReaderAddress", "J"); + if (fidNativeManager == nullptr) + { + return false; + } + + env->SetLongField(this_obj, fidNativeManager, (long)in_file); + } + + return ret.ok(); +} + +#ifdef __cplusplus +} +#endif diff --git a/java/README.md b/java/README.md index c69ff88ffa2..f3cd647c60a 100644 --- a/java/README.md +++ b/java/README.md @@ -45,6 +45,15 @@ mvn install -P gandiva -pl gandiva -am -Dgandiva.cpp.build.dir=../../debug This library is still in Alpha stages, and subject to API changes without deprecation warnings. +## Building and running tests for native orc (optional) +Arrow Cpp must be built before this step. The cpp build directory must +be provided as the value for argument arrow.cpp.build.dir. eg. + +``` +cd java +mvn install -P native-orc -pl adapter/orc -am -Dgandiva.cpp.build.dir=../../debug +``` + ## Java Code Style Guide Arrow Java follows the Google style guide [here][3] with the following diff --git a/java/adapter/orc/CMakeLists.txt b/java/adapter/orc/CMakeLists.txt new file mode 100644 index 00000000000..4eccf2cf9bd --- /dev/null +++ b/java/adapter/orc/CMakeLists.txt @@ -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. + +# +# arrow_orc_java +# + +# Headers: top level + +project(arrow_orc_java) + +# Find java/jni +include(FindJava) +include(UseJava) +include(FindJNI) + +message("generating headers to ${JNI_HEADERS_DIR}") + +add_jar( + arrow_orc_java + src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java + GENERATE_NATIVE_HEADERS arrow_orc_java-native + DESTINATION ${JNI_HEADERS_DIR} +) diff --git a/java/adapter/orc/pom.xml b/java/adapter/orc/pom.xml new file mode 100644 index 00000000000..515d6e05437 --- /dev/null +++ b/java/adapter/orc/pom.xml @@ -0,0 +1,54 @@ + + + + + 4.0.0 + + + org.apache.arrow + arrow-java-root + 0.14.0-SNAPSHOT + ../../pom.xml + + + org.apache.arrow.orc + arrow-orc + Arrow Orc Adapter + jar + + ../../../cpp/debug-build/ + + + + + + ${arrow.cpp.build.dir}/src/arrow/adapters/orc/jni/ + + **/libarrow_orc_jni.* + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + UTC + + + + + + diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java new file mode 100644 index 00000000000..9c6ee854806 --- /dev/null +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java @@ -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. + */ + +package org.apache.arrow.adapter.orc; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.InvalidPathException; +import java.nio.file.StandardCopyOption; +import java.util.UUID; + +public class OrcReaderJniWrapper { + private static final String LIBRARY_NAME = "arrow_orc_jni"; + + private long nativeReaderAddress; + + static { + try { + loadOrcAdapterLibraryFromJar(); + } catch (IOException e) { + throw new ExceptionInInitializerError(e); + } + } + + private static void loadOrcAdapterLibraryFromJar() + throws IOException { + final String libraryToLoad = System.mapLibraryName(LIBRARY_NAME); + final File libraryFile = moveFileFromJarToTemp( + System.getProperty("java.io.tmpdir"), libraryToLoad); + System.load(libraryFile.getAbsolutePath()); + } + + + private static File moveFileFromJarToTemp(final String tmpDir, String libraryToLoad) + throws IOException { + final File temp = setupFile(tmpDir, libraryToLoad); + try (final InputStream is = OrcReaderJniWrapper.class.getClassLoader() + .getResourceAsStream(libraryToLoad)) { + if (is == null) { + throw new InvalidPathException(libraryToLoad, "file was not found inside JAR."); + } else { + Files.copy(is, temp.toPath(), StandardCopyOption.REPLACE_EXISTING); + } + } + return temp; + } + + private static File setupFile(String tmpDir, String libraryToLoad) + throws IOException { + final String randomizeFileName = libraryToLoad + UUID.randomUUID(); + final File temp = new File(tmpDir, randomizeFileName); + if (temp.exists() && !temp.delete()) { + throw new InvalidPathException( + temp.getAbsolutePath(), "File already exists and cannot be removed."); + } + if (!temp.createNewFile()) { + throw new InvalidPathException(temp.getAbsolutePath(), "File could not be created."); + } + temp.deleteOnExit(); + return temp; + } + + private native boolean open(String fileName); + + public native void close(); + + public native boolean seek(int rowNumber); + + public native int getNumberOfStripes(); + + public native StripeReader nextStripeReader(long batchSize); + + public class StripeReader { + + private long nativeStripeReaderAddress; + + public native byte[] getSchema(); + + public native boolean next(int numRows, long[] outAddrs, long[] outSizes); + } +} diff --git a/java/pom.xml b/java/pom.xml index 666162571bd..dcea56302f6 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -682,6 +682,21 @@ gandiva + + + + native-orc + + format + memory + vector + tools + adapter/jdbc + adapter/orc + plasma + flight + + From e0d9c1f296438048d89288366d84aa2c78e1a332 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Mon, 20 May 2019 15:01:40 +0800 Subject: [PATCH 02/21] implement JNI interface on both size --- .../arrow/adapters/orc/jni/jni_wrapper.cpp | 230 ++++++++++++++++-- java/adapter/orc/CMakeLists.txt | 5 + java/adapter/orc/pom.xml | 14 ++ .../arrow/adapter/orc/OrcFieldNode.java | 37 +++ .../apache/arrow/adapter/orc/OrcJniUtils.java | 73 ++++++ .../adapter/orc/OrcMemoryJniWrapper.java | 63 +++++ .../apache/arrow/adapter/orc/OrcReader.java | 52 ++++ .../adapter/orc/OrcReaderJniWrapper.java | 60 +---- .../arrow/adapter/orc/OrcRecordBatch.java | 38 +++ .../adapter/orc/OrcReferenceManager.java | 208 ++++++++++++++++ .../arrow/adapter/orc/OrcStripeReader.java | 149 ++++++++++++ .../orc/OrcStripeReaderJniWrapper.java | 39 +++ .../arrow/memory/AllocationManager.java | 8 +- 13 files changed, 894 insertions(+), 82 deletions(-) create mode 100644 java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcFieldNode.java create mode 100644 java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java create mode 100644 java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java create mode 100644 java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java create mode 100644 java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java create mode 100644 java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java create mode 100644 java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java create mode 100644 java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java diff --git a/cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp b/cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp index 971535a4a2a..97ae5cc1876 100644 --- a/cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp +++ b/cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp @@ -15,43 +15,227 @@ // specific language governing permissions and limitations // under the License. -#include +#include +#include +#include #include +#include +#include +#include +#include +#include "org_apache_arrow_adapter_orc_OrcMemoryJniWrapper.h" #include "org_apache_arrow_adapter_orc_OrcReaderJniWrapper.h" -#include "org_apache_arrow_adapter_orc_OrcReaderJniWrapper_StripeReader.h" +#include "org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper.h" + +using ORCFileReader = arrow::adapters::orc::ORCFileReader; +using RecordBatchReader = arrow::RecordBatchReader; + +void ThrowJavaIOException(JNIEnv* env, const std::exception& e) { + jclass ioExceptionClass = env->FindClass("java/io/IOException"); + if (ioExceptionClass != nullptr) { + if (env->ThrowNew(ioExceptionClass, e.what())) { + // Failed to new IOException. This means another error has occurred in Java + // We just propagate this error to caller by doing nothing. + ARROW_LOG(ERROR) << "Error occurred when throwing IOException"; + } + } else { + ARROW_LOG(ERROR) << "Error occurred when getting IOException class"; + } +} + +void ThrowJavaException(JNIEnv* env, const std::string& message) { + jclass exception = env->FindClass("java/lang/Exception"); + if (exception != nullptr) { + env->ThrowNew(exception, message.c_str()); + } else { + throw std::runtime_error("Can't find java/lang/Exception class"); + } +} + +jfieldID GetFieldId(JNIEnv* env, jclass this_class, const std::string& sig) { + jfieldID ret = env->GetFieldID(this_class, sig.c_str(), "J"); + if (ret == nullptr) { + ThrowJavaException(env, "Unable to get java class field: " + sig); + } + + return ret; +} + +std::unique_ptr* GetNativeReader(JNIEnv* env, jobject this_obj) { + jlong reader = env->GetLongField( + this_obj, GetFieldId(env, env->GetObjectClass(this_obj), "nativeReaderAddress")); + return reinterpret_cast*>(reader); +} + +std::shared_ptr* GetStripeReader(JNIEnv* env, jobject this_obj) { + jlong reader = env->GetLongField( + this_obj, + GetFieldId(env, env->GetObjectClass(this_obj), "nativeStripeReaderAddress")); + return reinterpret_cast*>(reader); +} #ifdef __cplusplus extern "C" { #endif -JNIEXPORT jboolean JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_open - (JNIEnv *env, jobject this_obj, jstring file_path) { - auto in_file = new std::shared_ptr(); - const char *str= env->GetStringUTFChars(file_path, nullptr); - std::string path(str); - env->ReleaseStringUTFChars(file_path, str); - - arrow::Status ret; - if (path.find("hdfs://") == 0) { - return false; - } else { - ret = arrow::io::ReadableFile::Open(path, in_file); +JNIEXPORT jboolean JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_open( + JNIEnv* env, jobject this_obj, jstring file_path) { + std::shared_ptr in_file; + const char* str = env->GetStringUTFChars(file_path, nullptr); + std::string path(str); + env->ReleaseStringUTFChars(file_path, str); + + arrow::Status ret; + if (path.find("hdfs://") == 0) { + return false; + } else { + ret = arrow::io::ReadableFile::Open(path, &in_file); + } + + if (ret.ok()) { + auto reader = new std::unique_ptr(); + + ret = ORCFileReader::Open( + std::static_pointer_cast(in_file), + arrow::default_memory_pool(), reader); + + if (!ret.ok()) { + delete reader; + ThrowJavaIOException(env, std::invalid_argument("Failed open file" + path)); } - if(ret.ok()) { + env->SetLongField( + this_obj, GetFieldId(env, env->GetObjectClass(this_obj), "nativeReaderAddress"), + reinterpret_cast(reader)); + } + + return ret.ok(); +} + +JNIEXPORT void JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_close( + JNIEnv* env, jobject this_obj) { + delete GetNativeReader(env, this_obj); +} - jclass this_class = env->GetObjectClass(this_obj); - jfieldID fidNativeManager = env->GetFieldID(this_class, "nativeReaderAddress", "J"); - if (fidNativeManager == nullptr) - { - return false; - } +JNIEXPORT jboolean JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_seek( + JNIEnv* env, jobject this_obj, jint row_number) { + auto reader = GetNativeReader(env, this_obj); + return (*reader)->Seek(row_number).ok(); +} - env->SetLongField(this_obj, fidNativeManager, (long)in_file); +JNIEXPORT jint JNICALL +Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_getNumberOfStripes( + JNIEnv* env, jobject this_obj) { + auto reader = GetNativeReader(env, this_obj); + return (*reader)->NumberOfStripes(); +} + +JNIEXPORT jobject JNICALL +Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_nextStripeReader(JNIEnv* env, + jobject this_obj, + jlong batch_size) { + auto reader = GetNativeReader(env, this_obj); + auto stripe_reader = new std::shared_ptr(); + auto status = (*reader)->NextStripeReader(batch_size, stripe_reader); + + jobject ret = nullptr; + if (!status.ok()) { + delete stripe_reader; + return ret; + } + + jclass cls = env->FindClass("/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper"); + ret = env->AllocObject(cls); + + env->SetLongField(ret, GetFieldId(env, cls, "nativeStripeReaderAddress"), + reinterpret_cast(stripe_reader)); + + return ret; +} + +JNIEXPORT jbyteArray JNICALL +Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_getSchema(JNIEnv* env, + jobject this_obj) { + auto stripe_reader = GetStripeReader(env, this_obj); + auto schema = (*stripe_reader)->schema(); + + std::shared_ptr out; + auto status = arrow::ipc::SerializeSchema(*schema, arrow::default_memory_pool(), &out); + if (!status.ok()) { + return nullptr; + } + + jbyteArray ret = env->NewByteArray(out->size()); + memcpy(env->GetByteArrayElements(ret, nullptr), out->data(), out->size()); + return ret; +} + +JNIEXPORT jobject JNICALL +Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_next(JNIEnv* env, + jobject this_obj) { + auto stripe_reader = GetStripeReader(env, this_obj); + auto record_batch = new std::shared_ptr(); + auto status = (*stripe_reader)->ReadNext(record_batch); + if (!status.ok()) { + delete record_batch; + return nullptr; + } + + auto schema = (*stripe_reader)->schema(); + + // create OrcFieldNode[] + jclass field_class = env->FindClass("/org/apache/arrow/adapter/orc/OrcFieldNode"); + jmethodID field_constructor = env->GetMethodID(field_class, "", "(II)V"); + jobjectArray field_array = + env->NewObjectArray(schema->num_fields(), field_class, nullptr); + + std::vector> buffers; + for (int i = 0; i < schema->num_fields(); ++i) { + auto column = (*record_batch)->column(i); + auto dataArray = column->data(); + jobject field = env->NewObject(field_class, field_constructor, column->length(), + column->null_count()); + env->SetObjectArrayElement(field_array, i, field); + + for (auto& buffer : dataArray->buffers) { + buffers.push_back(buffer); } + } + + // create OrcMemoryJniWrapper[] + jclass memory_class = + env->FindClass("/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper"); + jmethodID memory_constructor = env->GetMethodID(memory_class, "", "(JJJJ)V"); + jobjectArray memory_array = env->NewObjectArray(buffers.size(), memory_class, nullptr); + + for (int j = 0; j < buffers.size(); ++j) { + auto buffer = buffers[j]; + jobject memory = env->NewObject(memory_class, memory_constructor, &buffer, + buffer->data(), buffer->size(), buffer->capacity()); + env->SetObjectArrayElement(memory_array, j, memory); + } + + // create OrcRecordBatch + jclass ret_cls = env->FindClass("/org/apache/arrow/adapter/orc/OrcRecordBatch"); + jmethodID ret_constructor = + env->GetMethodID(ret_cls, "", + "(I[L/org/apache/arrow/adapter/orc/OrcFieldNode;" + "[L/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper;)V"); + jobject ret = env->NewObject(ret_cls, ret_constructor, (*record_batch)->num_rows(), + field_array, memory_array); + + return ret; +} + +JNIEXPORT void JNICALL Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_close( + JNIEnv* env, jobject this_obj) { + delete GetStripeReader(env, this_obj); +} - return ret.ok(); +JNIEXPORT void JNICALL Java_org_apache_arrow_adapter_orc_OrcMemoryJniWrapper_release( + JNIEnv* env, jobject this_obj, jlong address) { + delete reinterpret_cast*>(address); } #ifdef __cplusplus diff --git a/java/adapter/orc/CMakeLists.txt b/java/adapter/orc/CMakeLists.txt index 4eccf2cf9bd..c6facacf465 100644 --- a/java/adapter/orc/CMakeLists.txt +++ b/java/adapter/orc/CMakeLists.txt @@ -33,6 +33,11 @@ message("generating headers to ${JNI_HEADERS_DIR}") add_jar( arrow_orc_java src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java + src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java + src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java + src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java + src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java + src/main/java/org/apache/arrow/adapter/orc/OrcFieldNode.java GENERATE_NATIVE_HEADERS arrow_orc_java-native DESTINATION ${JNI_HEADERS_DIR} ) diff --git a/java/adapter/orc/pom.xml b/java/adapter/orc/pom.xml index 515d6e05437..cbc7a468173 100644 --- a/java/adapter/orc/pom.xml +++ b/java/adapter/orc/pom.xml @@ -14,6 +14,20 @@ 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"> 4.0.0 + + + org.apache.arrow + arrow-memory + ${project.version} + compile + + + org.apache.arrow + arrow-vector + ${project.version} + compile + + org.apache.arrow diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcFieldNode.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcFieldNode.java new file mode 100644 index 00000000000..a38076ccd87 --- /dev/null +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcFieldNode.java @@ -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.arrow.adapter.orc; + +public class OrcFieldNode { + + private final int length; + private final int nullCount; + + public OrcFieldNode(int length, int nullCount) { + this.length = length; + this.nullCount = nullCount; + } + + public int getLength() { + return length; + } + + public int getNullCount() { + return nullCount; + } +} diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java new file mode 100644 index 00000000000..8c75e33ff71 --- /dev/null +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.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.arrow.adapter.orc; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.InvalidPathException; +import java.nio.file.StandardCopyOption; +import java.util.UUID; + +public class OrcJniUtils { + private static final String LIBRARY_NAME = "arrow_orc_jni"; + private static boolean isLoaded = false; + + public static void loadOrcAdapterLibraryFromJar() + throws IOException { + synchronized (OrcJniUtils.class) { + if (!isLoaded) { + final String libraryToLoad = System.mapLibraryName(LIBRARY_NAME); + final File libraryFile = moveFileFromJarToTemp( + System.getProperty("java.io.tmpdir"), libraryToLoad); + System.load(libraryFile.getAbsolutePath()); + isLoaded = true; + } + } + } + + private static File moveFileFromJarToTemp(final String tmpDir, String libraryToLoad) + throws IOException { + final File temp = setupFile(tmpDir, libraryToLoad); + try (final InputStream is = OrcReaderJniWrapper.class.getClassLoader() + .getResourceAsStream(libraryToLoad)) { + if (is == null) { + throw new InvalidPathException(libraryToLoad, "file was not found inside JAR."); + } else { + Files.copy(is, temp.toPath(), StandardCopyOption.REPLACE_EXISTING); + } + } + return temp; + } + + private static File setupFile(String tmpDir, String libraryToLoad) + throws IOException { + final String randomizeFileName = libraryToLoad + UUID.randomUUID(); + final File temp = new File(tmpDir, randomizeFileName); + if (temp.exists() && !temp.delete()) { + throw new InvalidPathException( + temp.getAbsolutePath(), "File already exists and cannot be removed."); + } + if (!temp.createNewFile()) { + throw new InvalidPathException(temp.getAbsolutePath(), "File could not be created."); + } + temp.deleteOnExit(); + return temp; + } +} diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java new file mode 100644 index 00000000000..24bfd0a6d7a --- /dev/null +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java @@ -0,0 +1,63 @@ +/* + * 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.arrow.adapter.orc; + +import java.io.IOException; + +public class OrcMemoryJniWrapper { + private final long ownershipAddress; + + private final long memoryAddress; + + private final long size; + + private final long capacity; + + static { + try { + OrcJniUtils.loadOrcAdapterLibraryFromJar(); + } catch (IOException e) { + throw new ExceptionInInitializerError(e); + } + } + + OrcMemoryJniWrapper(long ownershipAddress, long memoryAddress, long size, long capacity) { + this.ownershipAddress = ownershipAddress; + this.memoryAddress = memoryAddress; + this.size = size; + this.capacity = capacity; + } + + long getSize() { + return size; + } + + long getCapacity() { + return capacity; + } + + long getMemoryAddress() { + return memoryAddress; + } + + void release() { + release(ownershipAddress); + } + + private native void release(long address); +} diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java new file mode 100644 index 00000000000..f5aa6a36845 --- /dev/null +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java @@ -0,0 +1,52 @@ +/* + * 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.arrow.adapter.orc; + +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.vector.ipc.ArrowReader; + +import java.io.IOException; + +public class OrcReader implements AutoCloseable { + + private static final int DEFAULT_BATCH_SIZE = 1024; + + private OrcReaderJniWrapper orcReader; + + private BufferAllocator allocator; + + public OrcReader(String filePath, BufferAllocator allocator) throws IOException { + this.allocator = allocator; + orcReader = new OrcReaderJniWrapper(); + orcReader.open(filePath); + } + + public ArrowReader nextStripeReader() { + OrcStripeReaderJniWrapper stripeReader = orcReader.nextStripeReader(DEFAULT_BATCH_SIZE); + if (stripeReader == null) { + return null; + } + + return new OrcStripeReader(stripeReader, allocator); + } + + @Override + public void close() throws Exception { + orcReader.close(); + } +} diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java index 9c6ee854806..692eb61f56a 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java @@ -17,66 +17,21 @@ package org.apache.arrow.adapter.orc; -import java.io.File; import java.io.IOException; -import java.io.InputStream; -import java.nio.file.Files; -import java.nio.file.InvalidPathException; -import java.nio.file.StandardCopyOption; -import java.util.UUID; public class OrcReaderJniWrapper { - private static final String LIBRARY_NAME = "arrow_orc_jni"; private long nativeReaderAddress; static { try { - loadOrcAdapterLibraryFromJar(); + OrcJniUtils.loadOrcAdapterLibraryFromJar(); } catch (IOException e) { throw new ExceptionInInitializerError(e); } } - private static void loadOrcAdapterLibraryFromJar() - throws IOException { - final String libraryToLoad = System.mapLibraryName(LIBRARY_NAME); - final File libraryFile = moveFileFromJarToTemp( - System.getProperty("java.io.tmpdir"), libraryToLoad); - System.load(libraryFile.getAbsolutePath()); - } - - - private static File moveFileFromJarToTemp(final String tmpDir, String libraryToLoad) - throws IOException { - final File temp = setupFile(tmpDir, libraryToLoad); - try (final InputStream is = OrcReaderJniWrapper.class.getClassLoader() - .getResourceAsStream(libraryToLoad)) { - if (is == null) { - throw new InvalidPathException(libraryToLoad, "file was not found inside JAR."); - } else { - Files.copy(is, temp.toPath(), StandardCopyOption.REPLACE_EXISTING); - } - } - return temp; - } - - private static File setupFile(String tmpDir, String libraryToLoad) - throws IOException { - final String randomizeFileName = libraryToLoad + UUID.randomUUID(); - final File temp = new File(tmpDir, randomizeFileName); - if (temp.exists() && !temp.delete()) { - throw new InvalidPathException( - temp.getAbsolutePath(), "File already exists and cannot be removed."); - } - if (!temp.createNewFile()) { - throw new InvalidPathException(temp.getAbsolutePath(), "File could not be created."); - } - temp.deleteOnExit(); - return temp; - } - - private native boolean open(String fileName); + public native boolean open(String fileName); public native void close(); @@ -84,14 +39,5 @@ private static File setupFile(String tmpDir, String libraryToLoad) public native int getNumberOfStripes(); - public native StripeReader nextStripeReader(long batchSize); - - public class StripeReader { - - private long nativeStripeReaderAddress; - - public native byte[] getSchema(); - - public native boolean next(int numRows, long[] outAddrs, long[] outSizes); - } + public native OrcStripeReaderJniWrapper nextStripeReader(long batchSize); } diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java new file mode 100644 index 00000000000..e8fbf983a52 --- /dev/null +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java @@ -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.arrow.adapter.orc; + +import java.util.Arrays; +import java.util.List; + +public class OrcRecordBatch { + public final int length; + + /** + * Nodes correspond to the pre-ordered flattened logical schema. + */ + public final List nodes; + + public final List buffers; + + OrcRecordBatch(int length, OrcFieldNode[] nodes, OrcMemoryJniWrapper[] buffers) { + this.length = length; + this.nodes = Arrays.asList(nodes); + this.buffers = Arrays.asList(buffers); + } +} diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java new file mode 100644 index 00000000000..4d680ed6c1f --- /dev/null +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java @@ -0,0 +1,208 @@ +/* + * 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.arrow.adapter.orc; + +import io.netty.buffer.ArrowBuf; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.OwnershipTransferResult; +import org.apache.arrow.memory.ReferenceManager; +import org.apache.arrow.util.Preconditions; + +import java.util.concurrent.atomic.AtomicInteger; + +public class OrcReferenceManager implements ReferenceManager { + private final AtomicInteger bufRefCnt = new AtomicInteger(0); + + private OrcMemoryJniWrapper memory; + + OrcReferenceManager(OrcMemoryJniWrapper memory) { + this.memory = memory; + } + /** + * Return the reference count. + * + * @return reference count + */ + @Override + public int getRefCount() { + return bufRefCnt.get(); + } + + /** + * Decrement this reference manager's reference count by 1 for the associated underlying + * memory. If the reference count drops to 0, it implies that ArrowBufs managed by this + * reference manager no longer need access to the underlying memory + * + * @return true if ref count has dropped to 0, false otherwise + */ + @Override + public boolean release() { + return release(1); + } + + /** + * Decrement this reference manager's reference count for the associated underlying + * memory. If the reference count drops to 0, it implies that ArrowBufs managed by this + * reference manager no longer need access to the underlying memory + * + * @param decrement the count to decrease the reference count by + * @return the new reference count + */ + @Override + public boolean release(int decrement) { + Preconditions.checkState(decrement >= 1, + "ref count decrement should be greater than or equal to 1"); + // decrement the ref count + final int refCnt; + synchronized (this) { + refCnt = bufRefCnt.addAndGet(-decrement); + if (refCnt == 0) { + // refcount of this reference manager has dropped to 0 + // release the underlying memory + memory.release(); + } + } + // the new ref count should be >= 0 + Preconditions.checkState(refCnt >= 0, "RefCnt has gone negative"); + return refCnt == 0; + } + + /** + * Increment this reference manager's reference count by 1 for the associated underlying + * memory. + */ + @Override + public void retain() { + retain(1); + } + + /** + * Increment this reference manager's reference count by a given amount for the + * associated underlying memory. + * + * @param increment the count to increase the reference count by + */ + @Override + public void retain(int increment) { + Preconditions.checkArgument(increment > 0, "retain(%d) argument is not positive", increment); + final int originalReferenceCount = bufRefCnt.getAndAdd(increment); + Preconditions.checkArgument(originalReferenceCount > 0); + } + + /** + * Create a new ArrowBuf that is associated with an alternative allocator for the purposes of + * memory ownership and accounting. This has no impact on the reference counting for the current + * ArrowBuf except in the situation where the passed in Allocator is the same as the current buffer. + * This operation has no impact on the reference count of this ArrowBuf. The newly created + * ArrowBuf with either have a reference count of 1 (in the case that this is the first time this + * memory is being associated with the target allocator or in other words allocation manager currently + * doesn't hold a mapping for the target allocator) or the current value of the reference count for + * the target allocator-reference manager combination + 1 in the case that the provided allocator + * already had an association to this underlying memory. + * + * @param srcBuffer source ArrowBuf + * @param targetAllocator The target allocator to create an association with. + * @return A new ArrowBuf which shares the same underlying memory as this ArrowBuf. + */ + @Override + public ArrowBuf retain(ArrowBuf srcBuffer, BufferAllocator targetAllocator) { + retain(); + return srcBuffer; + } + + /** + * Derive a new ArrowBuf from a given source ArrowBuf. The new derived + * ArrowBuf will share the same reference count as rest of the ArrowBufs + * associated with this reference manager. + * + * @param sourceBuffer source ArrowBuf + * @param index index (relative to source ArrowBuf) new ArrowBuf should be derived from + * @param length length (bytes) of data in underlying memory that derived buffer will + * have access to in underlying memory + * @return derived buffer + */ + @Override + public ArrowBuf deriveBuffer(ArrowBuf sourceBuffer, int index, int length) { + final long derivedBufferAddress = sourceBuffer.memoryAddress() + index; + + // create new ArrowBuf + final ArrowBuf derivedBuf = new ArrowBuf( + this, + null, + length, // length (in bytes) in the underlying memory chunk for this new ArrowBuf + derivedBufferAddress, // starting byte address in the underlying memory for this new ArrowBuf, + false); + + return derivedBuf; + } + + /** + * Transfer the memory accounting ownership of this ArrowBuf to another allocator. + * This will generate a new ArrowBuf that carries an association with the underlying memory + * for the given ArrowBuf + * + * @param sourceBuffer source ArrowBuf + * @param targetAllocator The target allocator to create an association with + * @return {@link OwnershipTransferResult} with info on transfer result and new buffer + */ + @Override + public OwnershipTransferResult transferOwnership(ArrowBuf sourceBuffer, BufferAllocator targetAllocator) { + retain(); + return new OwnershipTransferResult() { + @Override + public boolean getAllocationFit() { + return false; + } + + @Override + public ArrowBuf getTransferredBuffer() { + return sourceBuffer; + } + }; + } + + /** + * Get the buffer allocator associated with this reference manager + * + * @return buffer allocator. + */ + @Override + public BufferAllocator getAllocator() { + return null; + } + + /** + * Total size (in bytes) of memory underlying this reference manager. + * + * @return Size (in bytes) of the memory chunk. + */ + @Override + public int getSize() { + return memory.getSize(); + } + + /** + * Get the total accounted size (in bytes). + * + * @return accounted size. + */ + @Override + public int getAccountedSize() { + return 0; + } +} diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java new file mode 100644 index 00000000000..424a88bba56 --- /dev/null +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java @@ -0,0 +1,149 @@ +/* + * 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.arrow.adapter.orc; + +import io.netty.buffer.ArrowBuf; +import org.apache.arrow.flatbuf.MessageHeader; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.vector.ipc.ArrowReader; +import org.apache.arrow.vector.ipc.ReadChannel; +import org.apache.arrow.vector.ipc.message.*; +import org.apache.arrow.vector.types.pojo.Schema; +import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.stream.Collectors; + +public class OrcStripeReader extends ArrowReader { + private OrcStripeReaderJniWrapper stripeReader; + private MessageChannelReader schemaReader; + + protected OrcStripeReader(OrcStripeReaderJniWrapper stripeReader, BufferAllocator allocator) { + super(allocator); + this.stripeReader = stripeReader; + } + + /** + * Load the next ArrowRecordBatch to the vector schema root if available. + * + * @return true if a batch was read, false on EOS + * @throws IOException on error + */ + @Override + public boolean loadNextBatch() throws IOException { + OrcRecordBatch recordBatch = stripeReader.next(); + if (recordBatch == null) { + return false; + } + + ArrayList buffers = new ArrayList<>(); + for(OrcMemoryJniWrapper buffer : recordBatch.buffers) { + buffers.add(new ArrowBuf( + new OrcReferenceManager(buffer), + null, + (int)buffer.getSize(), + buffer.getMemoryAddress(), + false)); + } + + loadRecordBatch(new ArrowRecordBatch( + recordBatch.length, + recordBatch.nodes.stream() + .map(buf -> new ArrowFieldNode(buf.getLength(), buf.getNullCount())) + .collect(Collectors.toList()), + buffers)); + return true; + } + + /** + * Return the number of bytes read from the ReadChannel. + * + * @return number of bytes read + */ + @Override + public long bytesRead() { + return 0; + } + + /** + * Close the underlying read source. + * + * @throws IOException on error + */ + @Override + protected void closeReadSource() throws IOException { + stripeReader.close(); + schemaReader.close(); + } + + /** + * Read the Schema from the source, will be invoked at the beginning the initialization. + * + * @return the read Schema + * @throws IOException on error + */ + @Override + protected Schema readSchema() throws IOException { + byte[] schemaBytes = stripeReader.getSchema(); + schemaReader = new MessageChannelReader( + new ReadChannel( + new ByteArrayReadableSeekableByteChannel(schemaBytes)), allocator); + + MessageResult result = schemaReader.readNext(); + + if (result == null) { + throw new IOException("Unexpected end of input. Missing schema."); + } + + if (result.getMessage().headerType() != MessageHeader.Schema) { + throw new IOException("Expected schema but header was " + result.getMessage().headerType()); + } + + return MessageSerializer.deserializeSchema(result.getMessage()); + } + + /** + * Read a dictionary batch from the source, will be invoked after the schema has been read and + * called N times, where N is the number of dictionaries indicated by the schema Fields. + * + * @return the read ArrowDictionaryBatch + * @throws IOException on error + */ + @Override + protected ArrowDictionaryBatch readDictionary() throws IOException { + MessageResult result = schemaReader.readNext(); + + if (result == null) { + throw new IOException("Unexpected end of input. Expected DictionaryBatch"); + } + + if (result.getMessage().headerType() != MessageHeader.DictionaryBatch) { + throw new IOException("Expected DictionaryBatch but header was " + result.getMessage().headerType()); + } + + ArrowBuf bodyBuffer = result.getBodyBuffer(); + + // For zero-length batches, need an empty buffer to deserialize the batch + if (bodyBuffer == null) { + bodyBuffer = allocator.getEmpty(); + } + + return MessageSerializer.deserializeDictionaryBatch(result.getMessage(), bodyBuffer); + } +} diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java new file mode 100644 index 00000000000..31d63951e85 --- /dev/null +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java @@ -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.arrow.adapter.orc; + +import java.io.IOException; + +public class OrcStripeReaderJniWrapper { + + static { + try { + OrcJniUtils.loadOrcAdapterLibraryFromJar(); + } catch (IOException e) { + throw new ExceptionInInitializerError(e); + } + } + + private long nativeStripeReaderAddress; + + public native byte[] getSchema(); + + public native OrcRecordBatch next(); + + public native void close(); +} diff --git a/java/memory/src/main/java/org/apache/arrow/memory/AllocationManager.java b/java/memory/src/main/java/org/apache/arrow/memory/AllocationManager.java index 993ae5d3766..825082eab0c 100644 --- a/java/memory/src/main/java/org/apache/arrow/memory/AllocationManager.java +++ b/java/memory/src/main/java/org/apache/arrow/memory/AllocationManager.java @@ -70,7 +70,7 @@ public class AllocationManager { private volatile BufferLedger owningLedger; private volatile long amDestructionTime = 0; - AllocationManager(BaseAllocator accountingAllocator, int size) { + public AllocationManager(BaseAllocator accountingAllocator, int size) { Preconditions.checkNotNull(accountingAllocator); accountingAllocator.assertOpen(); @@ -174,7 +174,7 @@ void release(final BufferLedger ledger) { // underlying memory chunk as it is no longer being referenced ((BaseAllocator)oldLedger.getAllocator()).releaseBytes(size); // free the memory chunk associated with the allocation manager - memoryChunk.release(); + releaseMemory(); ((BaseAllocator)oldLedger.getAllocator()).getListener().onRelease(size); amDestructionTime = System.nanoTime(); owningLedger = null; @@ -196,6 +196,10 @@ void release(final BufferLedger ledger) { } } + public void releaseMemory() { + memoryChunk.release(); + } + /** * Return the size of underlying chunk of memory managed by this Allocation Manager. * @return size of memory chunk From 1c0e0b2a5f69771f6f5b3a6827d13a597bbae669 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Mon, 20 May 2019 15:35:44 +0800 Subject: [PATCH 03/21] Fix minor build errors --- java/adapter/orc/pom.xml | 6 +++++ .../apache/arrow/adapter/orc/OrcReader.java | 3 ++- .../adapter/orc/OrcReferenceManager.java | 10 +++++---- .../arrow/adapter/orc/OrcStripeReader.java | 22 +++++++++++++------ 4 files changed, 29 insertions(+), 12 deletions(-) diff --git a/java/adapter/orc/pom.xml b/java/adapter/orc/pom.xml index cbc7a468173..28f7b7d7d67 100644 --- a/java/adapter/orc/pom.xml +++ b/java/adapter/orc/pom.xml @@ -27,6 +27,12 @@ ${project.version} compile + + org.apache.arrow + arrow-format + ${project.version} + compile + diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java index f5aa6a36845..1cbe30229f2 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java @@ -17,10 +17,11 @@ package org.apache.arrow.adapter.orc; +import java.io.IOException; + import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.vector.ipc.ArrowReader; -import java.io.IOException; public class OrcReader implements AutoCloseable { diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java index 4d680ed6c1f..9fdbfc3d55a 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java @@ -17,13 +17,14 @@ package org.apache.arrow.adapter.orc; -import io.netty.buffer.ArrowBuf; +import java.util.concurrent.atomic.AtomicInteger; + import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.memory.OwnershipTransferResult; import org.apache.arrow.memory.ReferenceManager; import org.apache.arrow.util.Preconditions; -import java.util.concurrent.atomic.AtomicInteger; +import io.netty.buffer.ArrowBuf; public class OrcReferenceManager implements ReferenceManager { private final AtomicInteger bufRefCnt = new AtomicInteger(0); @@ -33,6 +34,7 @@ public class OrcReferenceManager implements ReferenceManager { OrcReferenceManager(OrcMemoryJniWrapper memory) { this.memory = memory; } + /** * Return the reference count. * @@ -177,7 +179,7 @@ public ArrowBuf getTransferredBuffer() { } /** - * Get the buffer allocator associated with this reference manager + * Get the buffer allocator associated with this reference manager. * * @return buffer allocator. */ @@ -193,7 +195,7 @@ public BufferAllocator getAllocator() { */ @Override public int getSize() { - return memory.getSize(); + return (int)memory.getSize(); } /** diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java index 424a88bba56..1b8b0d13994 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java @@ -17,18 +17,25 @@ package org.apache.arrow.adapter.orc; -import io.netty.buffer.ArrowBuf; +import java.io.IOException; +import java.util.ArrayList; +import java.util.stream.Collectors; + import org.apache.arrow.flatbuf.MessageHeader; import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.vector.ipc.ArrowReader; import org.apache.arrow.vector.ipc.ReadChannel; -import org.apache.arrow.vector.ipc.message.*; +import org.apache.arrow.vector.ipc.message.ArrowDictionaryBatch; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; +import org.apache.arrow.vector.ipc.message.ArrowRecordBatch; +import org.apache.arrow.vector.ipc.message.MessageChannelReader; +import org.apache.arrow.vector.ipc.message.MessageResult; +import org.apache.arrow.vector.ipc.message.MessageSerializer; import org.apache.arrow.vector.types.pojo.Schema; import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel; -import java.io.IOException; -import java.util.ArrayList; -import java.util.stream.Collectors; +import io.netty.buffer.ArrowBuf; + public class OrcStripeReader extends ArrowReader { private OrcStripeReaderJniWrapper stripeReader; @@ -53,7 +60,7 @@ public boolean loadNextBatch() throws IOException { } ArrayList buffers = new ArrayList<>(); - for(OrcMemoryJniWrapper buffer : recordBatch.buffers) { + for (OrcMemoryJniWrapper buffer : recordBatch.buffers) { buffers.add(new ArrowBuf( new OrcReferenceManager(buffer), null, @@ -134,7 +141,8 @@ protected ArrowDictionaryBatch readDictionary() throws IOException { } if (result.getMessage().headerType() != MessageHeader.DictionaryBatch) { - throw new IOException("Expected DictionaryBatch but header was " + result.getMessage().headerType()); + throw new IOException("Expected DictionaryBatch but header was " + + result.getMessage().headerType()); } ArrowBuf bodyBuffer = result.getBodyBuffer(); From 3604c24d334a36a5d901febf47a3e19790c8023b Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Mon, 20 May 2019 15:48:09 +0800 Subject: [PATCH 04/21] Resolve merge conflicts --- cpp/cmake_modules/DefineOptions.cmake | 2 ++ 1 file changed, 2 insertions(+) diff --git a/cpp/cmake_modules/DefineOptions.cmake b/cpp/cmake_modules/DefineOptions.cmake index b00af80e898..e8e47cce951 100644 --- a/cpp/cmake_modules/DefineOptions.cmake +++ b/cpp/cmake_modules/DefineOptions.cmake @@ -151,6 +151,8 @@ if("${CMAKE_SOURCE_DIR}" STREQUAL "${CMAKE_CURRENT_SOURCE_DIR}") define_option(ARROW_ORC "Build the Arrow ORC adapter" OFF) + define_option(ARROW_ORC_JNI "Build the Arrow ORC JNI lib" OFF) + define_option(ARROW_TENSORFLOW "Build Arrow with TensorFlow support enabled" OFF) define_option(ARROW_JEMALLOC "Build the Arrow jemalloc-based allocator" ON) From 1b6a7042aa5f4d52b06b617a19868cf0627e681a Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Mon, 27 May 2019 14:41:32 +0800 Subject: [PATCH 05/21] Interface refactor and performance optimization - refactor JNI interface to make the underlying resource id explicit - add concurrent map for resource management - hold reference to class and method field to avoid unnecessary overhead --- .../arrow/adapters/orc/jni/concurrent_map.h | 80 ++++++ .../arrow/adapters/orc/jni/jni_wrapper.cpp | 249 +++++++++++------- java/README.md | 2 +- .../arrow/adapter/orc/OrcFieldNode.java | 14 +- .../apache/arrow/adapter/orc/OrcJniUtils.java | 11 +- .../adapter/orc/OrcMemoryJniWrapper.java | 25 +- .../apache/arrow/adapter/orc/OrcReader.java | 58 +++- .../adapter/orc/OrcReaderJniWrapper.java | 46 +++- .../arrow/adapter/orc/OrcRecordBatch.java | 14 +- .../adapter/orc/OrcReferenceManager.java | 90 +------ .../arrow/adapter/orc/OrcStripeReader.java | 76 ++---- .../orc/OrcStripeReaderJniWrapper.java | 32 ++- .../arrow/memory/AllocationManager.java | 8 +- 13 files changed, 413 insertions(+), 292 deletions(-) create mode 100644 cpp/src/arrow/adapters/orc/jni/concurrent_map.h diff --git a/cpp/src/arrow/adapters/orc/jni/concurrent_map.h b/cpp/src/arrow/adapters/orc/jni/concurrent_map.h new file mode 100644 index 00000000000..8763a6db6ca --- /dev/null +++ b/cpp/src/arrow/adapters/orc/jni/concurrent_map.h @@ -0,0 +1,80 @@ +/* + * 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 + */ + +#ifndef JNI_ID_TO_MODULE_MAP_H +#define JNI_ID_TO_MODULE_MAP_H + +#include +#include + +namespace arrow { + +template +class concurrentMap { +public: + concurrentMap() : module_id_(kInitModuleId) {} + + jlong Insert(HOLDER holder) { + mtx_.lock(); + jlong result = module_id_++; + map_.insert(std::pair(result, holder)); + mtx_.unlock(); + return result; + } + + void Erase(jlong module_id) { + mtx_.lock(); + map_.erase(module_id); + mtx_.unlock(); + } + + HOLDER Lookup(jlong module_id) { + HOLDER result = nullptr; + try { + result = map_.at(module_id); + } catch (const std::out_of_range &e) { + } + if (result != nullptr) { + return result; + } + mtx_.lock(); + try { + result = map_.at(module_id); + } catch (const std::out_of_range &e) { + } + mtx_.unlock(); + return result; + } + + void Clear() { + mtx_.lock(); + map_.clear(); + mtx_.unlock(); + } + +private: + static const int kInitModuleId = 4; + + long module_id_; + std::mutex mtx_; + // map from module ids returned to Java and module pointers + std::unordered_map map_; +}; + +} // namespace gandiva + +#endif // JNI_ID_TO_MODULE_MAP_H diff --git a/cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp b/cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp index 97ae5cc1876..469340fa8d2 100644 --- a/cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp +++ b/cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp @@ -28,140 +28,198 @@ #include "org_apache_arrow_adapter_orc_OrcReaderJniWrapper.h" #include "org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper.h" +#include "concurrent_map.h" + using ORCFileReader = arrow::adapters::orc::ORCFileReader; using RecordBatchReader = arrow::RecordBatchReader; -void ThrowJavaIOException(JNIEnv* env, const std::exception& e) { - jclass ioExceptionClass = env->FindClass("java/io/IOException"); - if (ioExceptionClass != nullptr) { - if (env->ThrowNew(ioExceptionClass, e.what())) { - // Failed to new IOException. This means another error has occurred in Java - // We just propagate this error to caller by doing nothing. - ARROW_LOG(ERROR) << "Error occurred when throwing IOException"; - } - } else { - ARROW_LOG(ERROR) << "Error occurred when getting IOException class"; - } -} +static jclass io_exception_class; +static jclass exception_class; -void ThrowJavaException(JNIEnv* env, const std::string& message) { - jclass exception = env->FindClass("java/lang/Exception"); - if (exception != nullptr) { - env->ThrowNew(exception, message.c_str()); - } else { - throw std::runtime_error("Can't find java/lang/Exception class"); - } +static jclass orc_field_node_class; +static jmethodID orc_field_node_constructor; + +static jclass orc_memory_class; +static jmethodID orc_memory_constructor; + +static jclass record_batch_class; +static jmethodID record_batch_constructor; + +static jint JNI_VERSION = JNI_VERSION_1_6; + +static arrow::concurrentMap> buffer_holder_; +static arrow::concurrentMap> orc_stripe_reader_holder_; +static arrow::concurrentMap> orc_file_reader_holder_; + +jclass CreateGlobalClassReference(JNIEnv* env, const char* class_name) { + jclass local_class = env->FindClass(class_name); + jclass global_class = (jclass)env->NewGlobalRef(local_class); + env->DeleteLocalRef(local_class); + return global_class; } -jfieldID GetFieldId(JNIEnv* env, jclass this_class, const std::string& sig) { - jfieldID ret = env->GetFieldID(this_class, sig.c_str(), "J"); +jmethodID GetMethodID(JNIEnv* env, jclass this_class, const char* name, const char* sig) { + jmethodID ret = env->GetMethodID(this_class, name, sig); if (ret == nullptr) { - ThrowJavaException(env, "Unable to get java class field: " + sig); + std::string error_message = "Unable to find method " + std::string(name) + + " within signature" + std::string(sig); + env->ThrowNew(exception_class, error_message.c_str()); } return ret; } -std::unique_ptr* GetNativeReader(JNIEnv* env, jobject this_obj) { - jlong reader = env->GetLongField( - this_obj, GetFieldId(env, env->GetObjectClass(this_obj), "nativeReaderAddress")); - return reinterpret_cast*>(reader); +jint JNI_OnLoad(JavaVM* vm, void* reserved) { + JNIEnv* env; + if (vm->GetEnv(reinterpret_cast(&env), JNI_VERSION) != JNI_OK) { + return JNI_ERR; + } + + io_exception_class = CreateGlobalClassReference(env, "java/io/IOException"); + exception_class = CreateGlobalClassReference(env, "java/lang/Exception"); + + orc_field_node_class = + CreateGlobalClassReference(env, "/org/apache/arrow/adapter/orc/OrcFieldNode"); + orc_field_node_constructor = GetMethodID(env, orc_field_node_class, "", "(II)V"); + + orc_memory_class = CreateGlobalClassReference( + env, "/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper"); + orc_memory_constructor = GetMethodID(env, orc_memory_class, "", "(JJJJ)V"); + + record_batch_class = + CreateGlobalClassReference(env, "/org/apache/arrow/adapter/orc/OrcRecordBatch"); + record_batch_constructor = GetMethodID(env, record_batch_class, "", + "(I[L/org/apache/arrow/adapter/orc/OrcFieldNode;" + "[L/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper;)V"); + + env->ExceptionDescribe(); + + return JNI_VERSION; +} + +void JNI_OnUnload(JavaVM* vm, void* reserved) { + JNIEnv* env; + vm->GetEnv(reinterpret_cast(&env), JNI_VERSION); + env->DeleteGlobalRef(io_exception_class); + env->DeleteGlobalRef(exception_class); + env->DeleteGlobalRef(orc_field_node_class); + env->DeleteGlobalRef(orc_memory_class); + env->DeleteGlobalRef(record_batch_class); + + buffer_holder_.Clear(); + orc_stripe_reader_holder_.Clear(); + orc_file_reader_holder_.Clear(); +} + +std::shared_ptr GetNativeReader(jlong id) { + return orc_file_reader_holder_.Lookup(id); +} + +std::shared_ptr GetStripeReader(jlong id) { + return orc_stripe_reader_holder_.Lookup(id); } -std::shared_ptr* GetStripeReader(JNIEnv* env, jobject this_obj) { - jlong reader = env->GetLongField( - this_obj, - GetFieldId(env, env->GetObjectClass(this_obj), "nativeStripeReaderAddress")); - return reinterpret_cast*>(reader); +int jstr_to_cstr(JNIEnv* env, jstring jstr, char* cstr, size_t cstr_len) { + int32_t jlen, clen; + + clen = env->GetStringUTFLength(jstr); + if (clen > (int32_t)cstr_len) return -ENAMETOOLONG; + jlen = env->GetStringLength(jstr); + env->GetStringUTFRegion(jstr, 0, jlen, cstr); + if (env->ExceptionCheck()) return -EIO; + return 0; +} + +std::string JStringToCString(JNIEnv* env, jstring string) { + int32_t jlen, clen; + clen = env->GetStringUTFLength(string); + jlen = env->GetStringLength(string); + std::vector buffer(clen); + env->GetStringUTFRegion(string, 0, jlen, buffer.data()); + return std::string(buffer.data(), clen); } #ifdef __cplusplus extern "C" { #endif -JNIEXPORT jboolean JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_open( - JNIEnv* env, jobject this_obj, jstring file_path) { +JNIEXPORT jlong JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_open( + JNIEnv* env, jclass this_cls, jstring file_path) { std::shared_ptr in_file; - const char* str = env->GetStringUTFChars(file_path, nullptr); - std::string path(str); - env->ReleaseStringUTFChars(file_path, str); + + std::string path = JStringToCString(env, file_path); arrow::Status ret; if (path.find("hdfs://") == 0) { - return false; + env->ThrowNew(io_exception_class, "hdfs path not support yet."); } else { ret = arrow::io::ReadableFile::Open(path, &in_file); } if (ret.ok()) { - auto reader = new std::unique_ptr(); + std::unique_ptr reader; ret = ORCFileReader::Open( std::static_pointer_cast(in_file), - arrow::default_memory_pool(), reader); + arrow::default_memory_pool(), &reader); if (!ret.ok()) { - delete reader; - ThrowJavaIOException(env, std::invalid_argument("Failed open file" + path)); + env->ThrowNew(io_exception_class, std::string("Failed open file" + path).c_str()); } - env->SetLongField( - this_obj, GetFieldId(env, env->GetObjectClass(this_obj), "nativeReaderAddress"), - reinterpret_cast(reader)); + return orc_file_reader_holder_.Insert( + std::shared_ptr(reader.release())); } - return ret.ok(); + return static_cast(ret.code()) * -1; } JNIEXPORT void JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_close( - JNIEnv* env, jobject this_obj) { - delete GetNativeReader(env, this_obj); + JNIEnv* env, jclass this_cls, jlong id) { + orc_file_reader_holder_.Erase(id); } JNIEXPORT jboolean JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_seek( - JNIEnv* env, jobject this_obj, jint row_number) { - auto reader = GetNativeReader(env, this_obj); - return (*reader)->Seek(row_number).ok(); + JNIEnv* env, jclass this_cls, jlong id, jint row_number) { + auto reader = orc_file_reader_holder_.Lookup(id); + return reader->Seek(row_number).ok(); } JNIEXPORT jint JNICALL -Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_getNumberOfStripes( - JNIEnv* env, jobject this_obj) { - auto reader = GetNativeReader(env, this_obj); - return (*reader)->NumberOfStripes(); +Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_getNumberOfStripes(JNIEnv* env, + jclass this_cls, + jlong id) { + auto reader = orc_file_reader_holder_.Lookup(id); + return reader->NumberOfStripes(); } -JNIEXPORT jobject JNICALL +JNIEXPORT jlong JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_nextStripeReader(JNIEnv* env, - jobject this_obj, + jclass this_cls, + jlong id, jlong batch_size) { - auto reader = GetNativeReader(env, this_obj); - auto stripe_reader = new std::shared_ptr(); - auto status = (*reader)->NextStripeReader(batch_size, stripe_reader); + auto reader = GetNativeReader(id); + std::shared_ptr stripe_reader; + auto status = reader->NextStripeReader(batch_size, &stripe_reader); - jobject ret = nullptr; if (!status.ok()) { - delete stripe_reader; - return ret; + return static_cast(status.code()) * -1; + ; } - jclass cls = env->FindClass("/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper"); - ret = env->AllocObject(cls); - - env->SetLongField(ret, GetFieldId(env, cls, "nativeStripeReaderAddress"), - reinterpret_cast(stripe_reader)); - - return ret; + return orc_stripe_reader_holder_.Insert(stripe_reader); } JNIEXPORT jbyteArray JNICALL Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_getSchema(JNIEnv* env, - jobject this_obj) { - auto stripe_reader = GetStripeReader(env, this_obj); - auto schema = (*stripe_reader)->schema(); + jclass this_cls, + jlong id) { + auto stripe_reader = GetStripeReader(id); + auto schema = stripe_reader->schema(); std::shared_ptr out; - auto status = arrow::ipc::SerializeSchema(*schema, arrow::default_memory_pool(), &out); + auto status = + arrow::ipc::SerializeSchema(*schema, nullptr, arrow::default_memory_pool(), &out); if (!status.ok()) { return nullptr; } @@ -173,29 +231,28 @@ Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_getSchema(JNIEnv* en JNIEXPORT jobject JNICALL Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_next(JNIEnv* env, - jobject this_obj) { - auto stripe_reader = GetStripeReader(env, this_obj); + jclass this_cls, + jlong id) { + auto stripe_reader = GetStripeReader(id); auto record_batch = new std::shared_ptr(); - auto status = (*stripe_reader)->ReadNext(record_batch); + auto status = stripe_reader->ReadNext(record_batch); if (!status.ok()) { delete record_batch; return nullptr; } - auto schema = (*stripe_reader)->schema(); + auto schema = stripe_reader->schema(); // create OrcFieldNode[] - jclass field_class = env->FindClass("/org/apache/arrow/adapter/orc/OrcFieldNode"); - jmethodID field_constructor = env->GetMethodID(field_class, "", "(II)V"); jobjectArray field_array = - env->NewObjectArray(schema->num_fields(), field_class, nullptr); + env->NewObjectArray(schema->num_fields(), orc_field_node_class, nullptr); std::vector> buffers; for (int i = 0; i < schema->num_fields(); ++i) { auto column = (*record_batch)->column(i); auto dataArray = column->data(); - jobject field = env->NewObject(field_class, field_constructor, column->length(), - column->null_count()); + jobject field = env->NewObject(orc_field_node_class, orc_field_node_constructor, + column->length(), column->null_count()); env->SetObjectArrayElement(field_array, i, field); for (auto& buffer : dataArray->buffers) { @@ -204,38 +261,32 @@ Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_next(JNIEnv* env, } // create OrcMemoryJniWrapper[] - jclass memory_class = - env->FindClass("/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper"); - jmethodID memory_constructor = env->GetMethodID(memory_class, "", "(JJJJ)V"); - jobjectArray memory_array = env->NewObjectArray(buffers.size(), memory_class, nullptr); + jobjectArray memory_array = + env->NewObjectArray(buffers.size(), orc_memory_class, nullptr); for (int j = 0; j < buffers.size(); ++j) { auto buffer = buffers[j]; - jobject memory = env->NewObject(memory_class, memory_constructor, &buffer, - buffer->data(), buffer->size(), buffer->capacity()); + jobject memory = env->NewObject(orc_memory_class, orc_memory_constructor, + buffer_holder_.Insert(buffer), buffer->data(), + buffer->size(), buffer->capacity()); env->SetObjectArrayElement(memory_array, j, memory); } // create OrcRecordBatch - jclass ret_cls = env->FindClass("/org/apache/arrow/adapter/orc/OrcRecordBatch"); - jmethodID ret_constructor = - env->GetMethodID(ret_cls, "", - "(I[L/org/apache/arrow/adapter/orc/OrcFieldNode;" - "[L/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper;)V"); - jobject ret = env->NewObject(ret_cls, ret_constructor, (*record_batch)->num_rows(), - field_array, memory_array); + jobject ret = env->NewObject(record_batch_class, record_batch_constructor, + (*record_batch)->num_rows(), field_array, memory_array); return ret; } JNIEXPORT void JNICALL Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_close( - JNIEnv* env, jobject this_obj) { - delete GetStripeReader(env, this_obj); + JNIEnv* env, jclass this_cls, jlong id) { + buffer_holder_.Erase(id); } JNIEXPORT void JNICALL Java_org_apache_arrow_adapter_orc_OrcMemoryJniWrapper_release( - JNIEnv* env, jobject this_obj, jlong address) { - delete reinterpret_cast*>(address); + JNIEnv* env, jobject this_obj, jlong id) { + buffer_holder_.Erase(id); } #ifdef __cplusplus diff --git a/java/README.md b/java/README.md index f3cd647c60a..df918a2bf98 100644 --- a/java/README.md +++ b/java/README.md @@ -51,7 +51,7 @@ be provided as the value for argument arrow.cpp.build.dir. eg. ``` cd java -mvn install -P native-orc -pl adapter/orc -am -Dgandiva.cpp.build.dir=../../debug +mvn install -P native-orc -pl adapter/orc -am -Darrow.cpp.build.dir=../../debug ``` ## Java Code Style Guide diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcFieldNode.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcFieldNode.java index a38076ccd87..aedbd61816b 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcFieldNode.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcFieldNode.java @@ -17,21 +17,29 @@ package org.apache.arrow.adapter.orc; -public class OrcFieldNode { +/** + * Metadata about Vectors/Arrays that is passed via JNI interface + */ +class OrcFieldNode { private final int length; private final int nullCount; + /** + * Construct a new instance. + * @param length the number of values written. + * @param nullCount the number of null values. + */ public OrcFieldNode(int length, int nullCount) { this.length = length; this.nullCount = nullCount; } - public int getLength() { + int getLength() { return length; } - public int getNullCount() { + int getNullCount() { return nullCount; } } diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java index 8c75e33ff71..c14cb7454e2 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java @@ -25,11 +25,18 @@ import java.nio.file.StandardCopyOption; import java.util.UUID; -public class OrcJniUtils { +/** + * Helper class for JNI related operations. + */ +class OrcJniUtils { private static final String LIBRARY_NAME = "arrow_orc_jni"; private static boolean isLoaded = false; - public static void loadOrcAdapterLibraryFromJar() + /** + * Load arrow orc jni library from jar. + * @throws IOException throws IOException in case target library can not be found. + */ + static void loadOrcAdapterLibraryFromJar() throws IOException { synchronized (OrcJniUtils.class) { if (!isLoaded) { diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java index 24bfd0a6d7a..6e3ab3634f5 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java @@ -19,8 +19,11 @@ import java.io.IOException; -public class OrcMemoryJniWrapper { - private final long ownershipAddress; +/** + * Wrapper for orc memory allocated by native code. + */ +class OrcMemoryJniWrapper implements AutoCloseable { + private final long id; private final long memoryAddress; @@ -36,8 +39,15 @@ public class OrcMemoryJniWrapper { } } - OrcMemoryJniWrapper(long ownershipAddress, long memoryAddress, long size, long capacity) { - this.ownershipAddress = ownershipAddress; + /** + * Construct a new instance. + * @param id unique id of the underlying memory. + * @param memoryAddress starting memory address of the the underlying memory. + * @param size size of the valid data. + * @param capacity allocated memory size. + */ + OrcMemoryJniWrapper(long id, long memoryAddress, long size, long capacity) { + this.id = id; this.memoryAddress = memoryAddress; this.size = size; this.capacity = capacity; @@ -55,9 +65,10 @@ long getMemoryAddress() { return memoryAddress; } - void release() { - release(ownershipAddress); + @Override + public void close() { + release(id); } - private native void release(long address); + private native void release(long id); } diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java index 1cbe30229f2..e0bd1a5a692 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java @@ -22,32 +22,64 @@ import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.vector.ipc.ArrowReader; - +/** + * Orc Reader that allow accessing orc stripes in Orc file. + * This orc reader basically acts like an ArrowReader iterator that + * iterate over orc stripes. Each stripe will be accessed via an + * ArrowReader. + */ public class OrcReader implements AutoCloseable { - - private static final int DEFAULT_BATCH_SIZE = 1024; - - private OrcReaderJniWrapper orcReader; - private BufferAllocator allocator; + private final long id; + + /** + * Create an OrcReader that iterate over orc stripes. + * @param filePath file path to target file, currently only support local file. + * @param allocator allocator provided to ArrowReader. + * @throws IOException throws exception in case of file not found + */ public OrcReader(String filePath, BufferAllocator allocator) throws IOException { this.allocator = allocator; - orcReader = new OrcReaderJniWrapper(); - orcReader.open(filePath); + id = OrcReaderJniWrapper.open(filePath); + } + + /** + * Seek to designated row. Invoke NextStripeReader() after seek + * will return stripe reader starting from designated row. + * @param rowNumber the rows number to seek + * @return true if seek operation is succeeded + */ + public boolean seek(int rowNumber) { + return OrcReaderJniWrapper.seek(id, rowNumber); } - public ArrowReader nextStripeReader() { - OrcStripeReaderJniWrapper stripeReader = orcReader.nextStripeReader(DEFAULT_BATCH_SIZE); - if (stripeReader == null) { + /** + * Get a stripe level ArrowReader with specified batchSize in each record batch. + * + * @param batchSize the number of rows loaded on each iteration + * @return ArrowReader that iterate over current stripes + */ + public ArrowReader nextStripeReader(long batchSize) { + long stripeReaderId = OrcReaderJniWrapper.nextStripeReader(id, batchSize); + if (stripeReaderId < 0) { return null; } - return new OrcStripeReader(stripeReader, allocator); + return new OrcStripeReader(stripeReaderId, allocator); + } + + /** + * The number of stripes in the file. + * + * @return number of stripes + */ + public int getNumberOfStripes() { + return OrcReaderJniWrapper.getNumberOfStripes(id); } @Override public void close() throws Exception { - orcReader.close(); + OrcReaderJniWrapper.close(id); } } diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java index 692eb61f56a..eb888f31f0a 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java @@ -19,10 +19,10 @@ import java.io.IOException; -public class OrcReaderJniWrapper { - - private long nativeReaderAddress; - +/** + * JNI wrapper for Orc reader + */ +class OrcReaderJniWrapper { static { try { OrcJniUtils.loadOrcAdapterLibraryFromJar(); @@ -31,13 +31,41 @@ public class OrcReaderJniWrapper { } } - public native boolean open(String fileName); + /** + * Construct a orc file reader over the target file + * @param fileName absolute file path of target file + * @return id of the orc reader instance if file opened successfully, + * otherwise return error code * -1. + */ + static native long open(String fileName); - public native void close(); + /** + * Close the underlying reader and release related resources. + * @param readerId id of the reader instance. + */ + static native void close(long readerId); - public native boolean seek(int rowNumber); + /** + * Seek to designated row. Invoke nextStripeReader() after seek + * will return id of stripe reader starting from designated row. + * @param readerId id of the reader instance + * @param rowNumber the rows number to seek + * @return true if seek operation is succeeded + */ + static native boolean seek(long readerId, int rowNumber); - public native int getNumberOfStripes(); + /** + * The number of stripes in the file. + * @param readerId id of the reader instance + * @return number of stripes + */ + static native int getNumberOfStripes(long readerId); - public native OrcStripeReaderJniWrapper nextStripeReader(long batchSize); + /** + * Get a stripe level ArrowReader with specified batchSize in each record batch. + * @param readerId id of the reader instance + * @param batchSize the number of rows loaded on each iteration + * @return id of the stripe reader instance. + */ + static native long nextStripeReader(long readerId, long batchSize); } diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java index e8fbf983a52..c21616a433a 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java @@ -20,16 +20,22 @@ import java.util.Arrays; import java.util.List; -public class OrcRecordBatch { - public final int length; +class OrcRecordBatch { + final int length; /** * Nodes correspond to the pre-ordered flattened logical schema. */ - public final List nodes; + final List nodes; - public final List buffers; + final List buffers; + /** + * Construct a new instance + * @param length number of records included in current batch + * @param nodes meta data for each fields + * @param buffers buffers for underlying data + */ OrcRecordBatch(int length, OrcFieldNode[] nodes, OrcMemoryJniWrapper[] buffers) { this.length = length; this.nodes = Arrays.asList(nodes); diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java index 9fdbfc3d55a..476d23e7cd5 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java @@ -17,6 +17,8 @@ package org.apache.arrow.adapter.orc; +import io.netty.buffer.ArrowBuf; + import java.util.concurrent.atomic.AtomicInteger; import org.apache.arrow.memory.BufferAllocator; @@ -24,8 +26,10 @@ import org.apache.arrow.memory.ReferenceManager; import org.apache.arrow.util.Preconditions; -import io.netty.buffer.ArrowBuf; - +/** + * A simple reference manager implementation for memory allocated by native code. + * The underlying memory will be released when reference count reach zero. + */ public class OrcReferenceManager implements ReferenceManager { private final AtomicInteger bufRefCnt = new AtomicInteger(0); @@ -35,36 +39,16 @@ public class OrcReferenceManager implements ReferenceManager { this.memory = memory; } - /** - * Return the reference count. - * - * @return reference count - */ @Override public int getRefCount() { return bufRefCnt.get(); } - /** - * Decrement this reference manager's reference count by 1 for the associated underlying - * memory. If the reference count drops to 0, it implies that ArrowBufs managed by this - * reference manager no longer need access to the underlying memory - * - * @return true if ref count has dropped to 0, false otherwise - */ @Override public boolean release() { return release(1); } - /** - * Decrement this reference manager's reference count for the associated underlying - * memory. If the reference count drops to 0, it implies that ArrowBufs managed by this - * reference manager no longer need access to the underlying memory - * - * @param decrement the count to decrease the reference count by - * @return the new reference count - */ @Override public boolean release(int decrement) { Preconditions.checkState(decrement >= 1, @@ -76,7 +60,7 @@ public boolean release(int decrement) { if (refCnt == 0) { // refcount of this reference manager has dropped to 0 // release the underlying memory - memory.release(); + memory.close(); } } // the new ref count should be >= 0 @@ -84,21 +68,11 @@ public boolean release(int decrement) { return refCnt == 0; } - /** - * Increment this reference manager's reference count by 1 for the associated underlying - * memory. - */ @Override public void retain() { retain(1); } - /** - * Increment this reference manager's reference count by a given amount for the - * associated underlying memory. - * - * @param increment the count to increase the reference count by - */ @Override public void retain(int increment) { Preconditions.checkArgument(increment > 0, "retain(%d) argument is not positive", increment); @@ -106,38 +80,12 @@ public void retain(int increment) { Preconditions.checkArgument(originalReferenceCount > 0); } - /** - * Create a new ArrowBuf that is associated with an alternative allocator for the purposes of - * memory ownership and accounting. This has no impact on the reference counting for the current - * ArrowBuf except in the situation where the passed in Allocator is the same as the current buffer. - * This operation has no impact on the reference count of this ArrowBuf. The newly created - * ArrowBuf with either have a reference count of 1 (in the case that this is the first time this - * memory is being associated with the target allocator or in other words allocation manager currently - * doesn't hold a mapping for the target allocator) or the current value of the reference count for - * the target allocator-reference manager combination + 1 in the case that the provided allocator - * already had an association to this underlying memory. - * - * @param srcBuffer source ArrowBuf - * @param targetAllocator The target allocator to create an association with. - * @return A new ArrowBuf which shares the same underlying memory as this ArrowBuf. - */ @Override public ArrowBuf retain(ArrowBuf srcBuffer, BufferAllocator targetAllocator) { retain(); return srcBuffer; } - /** - * Derive a new ArrowBuf from a given source ArrowBuf. The new derived - * ArrowBuf will share the same reference count as rest of the ArrowBufs - * associated with this reference manager. - * - * @param sourceBuffer source ArrowBuf - * @param index index (relative to source ArrowBuf) new ArrowBuf should be derived from - * @param length length (bytes) of data in underlying memory that derived buffer will - * have access to in underlying memory - * @return derived buffer - */ @Override public ArrowBuf deriveBuffer(ArrowBuf sourceBuffer, int index, int length) { final long derivedBufferAddress = sourceBuffer.memoryAddress() + index; @@ -153,15 +101,6 @@ public ArrowBuf deriveBuffer(ArrowBuf sourceBuffer, int index, int length) { return derivedBuf; } - /** - * Transfer the memory accounting ownership of this ArrowBuf to another allocator. - * This will generate a new ArrowBuf that carries an association with the underlying memory - * for the given ArrowBuf - * - * @param sourceBuffer source ArrowBuf - * @param targetAllocator The target allocator to create an association with - * @return {@link OwnershipTransferResult} with info on transfer result and new buffer - */ @Override public OwnershipTransferResult transferOwnership(ArrowBuf sourceBuffer, BufferAllocator targetAllocator) { retain(); @@ -178,31 +117,16 @@ public ArrowBuf getTransferredBuffer() { }; } - /** - * Get the buffer allocator associated with this reference manager. - * - * @return buffer allocator. - */ @Override public BufferAllocator getAllocator() { return null; } - /** - * Total size (in bytes) of memory underlying this reference manager. - * - * @return Size (in bytes) of the memory chunk. - */ @Override public int getSize() { return (int)memory.getSize(); } - /** - * Get the total accounted size (in bytes). - * - * @return accounted size. - */ @Override public int getAccountedSize() { return 0; diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java index 1b8b0d13994..5594d488045 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java @@ -17,6 +17,8 @@ package org.apache.arrow.adapter.orc; +import io.netty.buffer.ArrowBuf; + import java.io.IOException; import java.util.ArrayList; import java.util.stream.Collectors; @@ -33,28 +35,28 @@ import org.apache.arrow.vector.ipc.message.MessageSerializer; import org.apache.arrow.vector.types.pojo.Schema; import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel; +import sun.reflect.generics.reflectiveObjects.NotImplementedException; -import io.netty.buffer.ArrowBuf; - - +/** + * Orc stripe that load data into ArrowRecordBatch. + */ public class OrcStripeReader extends ArrowReader { - private OrcStripeReaderJniWrapper stripeReader; + private final long id; private MessageChannelReader schemaReader; - protected OrcStripeReader(OrcStripeReaderJniWrapper stripeReader, BufferAllocator allocator) { + /** + * Construct a new instance. + * @param id id of the stripe reader instance + * @param allocator memory allocator for accounting. + */ + OrcStripeReader(long id, BufferAllocator allocator) { super(allocator); - this.stripeReader = stripeReader; + this.id = id; } - /** - * Load the next ArrowRecordBatch to the vector schema root if available. - * - * @return true if a batch was read, false on EOS - * @throws IOException on error - */ @Override public boolean loadNextBatch() throws IOException { - OrcRecordBatch recordBatch = stripeReader.next(); + OrcRecordBatch recordBatch = OrcStripeReaderJniWrapper.next(id); if (recordBatch == null) { return false; } @@ -78,36 +80,21 @@ public boolean loadNextBatch() throws IOException { return true; } - /** - * Return the number of bytes read from the ReadChannel. - * - * @return number of bytes read - */ @Override public long bytesRead() { return 0; } - /** - * Close the underlying read source. - * - * @throws IOException on error - */ + @Override protected void closeReadSource() throws IOException { - stripeReader.close(); + OrcStripeReaderJniWrapper.close(id); schemaReader.close(); } - /** - * Read the Schema from the source, will be invoked at the beginning the initialization. - * - * @return the read Schema - * @throws IOException on error - */ @Override protected Schema readSchema() throws IOException { - byte[] schemaBytes = stripeReader.getSchema(); + byte[] schemaBytes = OrcStripeReaderJniWrapper.getSchema(id); schemaReader = new MessageChannelReader( new ReadChannel( new ByteArrayReadableSeekableByteChannel(schemaBytes)), allocator); @@ -125,33 +112,8 @@ protected Schema readSchema() throws IOException { return MessageSerializer.deserializeSchema(result.getMessage()); } - /** - * Read a dictionary batch from the source, will be invoked after the schema has been read and - * called N times, where N is the number of dictionaries indicated by the schema Fields. - * - * @return the read ArrowDictionaryBatch - * @throws IOException on error - */ @Override protected ArrowDictionaryBatch readDictionary() throws IOException { - MessageResult result = schemaReader.readNext(); - - if (result == null) { - throw new IOException("Unexpected end of input. Expected DictionaryBatch"); - } - - if (result.getMessage().headerType() != MessageHeader.DictionaryBatch) { - throw new IOException("Expected DictionaryBatch but header was " + - result.getMessage().headerType()); - } - - ArrowBuf bodyBuffer = result.getBodyBuffer(); - - // For zero-length batches, need an empty buffer to deserialize the batch - if (bodyBuffer == null) { - bodyBuffer = allocator.getEmpty(); - } - - return MessageSerializer.deserializeDictionaryBatch(result.getMessage(), bodyBuffer); + throw new NotImplementedException(); } } diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java index 31d63951e85..02474b3234c 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java @@ -19,7 +19,10 @@ import java.io.IOException; -public class OrcStripeReaderJniWrapper { +/** + * JNI wrapper for orc stripe reader. + */ +class OrcStripeReaderJniWrapper { static { try { @@ -29,11 +32,24 @@ public class OrcStripeReaderJniWrapper { } } - private long nativeStripeReaderAddress; - - public native byte[] getSchema(); - - public native OrcRecordBatch next(); - - public native void close(); + /** + * Get the schema of current stripe. + * @param readerId id of the stripe reader instance. + * @return serialized schema. + */ + static native byte[] getSchema(long readerId); + + /** + * Load next record batch. + * @param readerId id of the stripe reader instance. + * @return loaded record batch, return null when reached + * the end of current stripe. + */ + static native OrcRecordBatch next(long readerId); + + /** + * Release resources of underlying reader. + * @param readerId id of the stripe reader instance. + */ + static native void close(long readerId); } diff --git a/java/memory/src/main/java/org/apache/arrow/memory/AllocationManager.java b/java/memory/src/main/java/org/apache/arrow/memory/AllocationManager.java index 825082eab0c..993ae5d3766 100644 --- a/java/memory/src/main/java/org/apache/arrow/memory/AllocationManager.java +++ b/java/memory/src/main/java/org/apache/arrow/memory/AllocationManager.java @@ -70,7 +70,7 @@ public class AllocationManager { private volatile BufferLedger owningLedger; private volatile long amDestructionTime = 0; - public AllocationManager(BaseAllocator accountingAllocator, int size) { + AllocationManager(BaseAllocator accountingAllocator, int size) { Preconditions.checkNotNull(accountingAllocator); accountingAllocator.assertOpen(); @@ -174,7 +174,7 @@ void release(final BufferLedger ledger) { // underlying memory chunk as it is no longer being referenced ((BaseAllocator)oldLedger.getAllocator()).releaseBytes(size); // free the memory chunk associated with the allocation manager - releaseMemory(); + memoryChunk.release(); ((BaseAllocator)oldLedger.getAllocator()).getListener().onRelease(size); amDestructionTime = System.nanoTime(); owningLedger = null; @@ -196,10 +196,6 @@ void release(final BufferLedger ledger) { } } - public void releaseMemory() { - memoryChunk.release(); - } - /** * Return the size of underlying chunk of memory managed by this Allocation Manager. * @return size of memory chunk From e932aa8b11c0c6d956a1c419f7f2ccc3e3df2487 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Mon, 27 May 2019 17:59:17 +0800 Subject: [PATCH 06/21] Move jni code to src/jni and change build flag to arrow_jni --- cpp/CMakeLists.txt | 8 ++++--- cpp/cmake_modules/DefineOptions.cmake | 2 +- cpp/src/arrow/adapters/orc/CMakeLists.txt | 4 ---- cpp/src/jni/CMakeLists.txt | 24 +++++++++++++++++++ .../orc/jni => jni/orc}/CMakeLists.txt | 4 ++-- .../orc/jni => jni/orc}/concurrent_map.h | 0 .../orc/jni => jni/orc}/jni_wrapper.cpp | 18 +++++++------- java/adapter/orc/pom.xml | 2 +- .../arrow/adapter/orc/OrcFieldNode.java | 2 +- .../adapter/orc/OrcReaderJniWrapper.java | 6 ++--- .../arrow/adapter/orc/OrcRecordBatch.java | 2 +- .../adapter/orc/OrcReferenceManager.java | 4 ++-- .../arrow/adapter/orc/OrcStripeReader.java | 7 +++--- .../orc/OrcStripeReaderJniWrapper.java | 2 +- 14 files changed, 53 insertions(+), 32 deletions(-) create mode 100644 cpp/src/jni/CMakeLists.txt rename cpp/src/{arrow/adapters/orc/jni => jni/orc}/CMakeLists.txt (92%) rename cpp/src/{arrow/adapters/orc/jni => jni/orc}/concurrent_map.h (100%) rename cpp/src/{arrow/adapters/orc/jni => jni/orc}/jni_wrapper.cpp (96%) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 740948c43d7..501c541e175 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -252,13 +252,11 @@ endif() if(MSVC) # ORC doesn't build on windows set(ARROW_ORC OFF) - set(ARROW_ORC_JNI OFF) # Plasma using glog is not fully tested on windows. set(ARROW_USE_GLOG OFF) endif() -if(ARROW_ORC_JNI) - set(ARROW_ORC ON) +if(ARROW_JNI) set(ARROW_BUILD_STATIC ON) endif() @@ -735,6 +733,10 @@ if(ARROW_PARQUET) endif() endif() +if(ARROW_JNI) + add_subdirectory(src/jni) +endif() + if(ARROW_GANDIVA) add_subdirectory(src/gandiva) endif() diff --git a/cpp/cmake_modules/DefineOptions.cmake b/cpp/cmake_modules/DefineOptions.cmake index e8e47cce951..d041da5ac40 100644 --- a/cpp/cmake_modules/DefineOptions.cmake +++ b/cpp/cmake_modules/DefineOptions.cmake @@ -151,7 +151,7 @@ if("${CMAKE_SOURCE_DIR}" STREQUAL "${CMAKE_CURRENT_SOURCE_DIR}") define_option(ARROW_ORC "Build the Arrow ORC adapter" OFF) - define_option(ARROW_ORC_JNI "Build the Arrow ORC JNI lib" OFF) + define_option(ARROW_JNI "Build the Arrow JNI lib" OFF) define_option(ARROW_TENSORFLOW "Build Arrow with TensorFlow support enabled" OFF) diff --git a/cpp/src/arrow/adapters/orc/CMakeLists.txt b/cpp/src/arrow/adapters/orc/CMakeLists.txt index c17b4091eda..8a29b17d602 100644 --- a/cpp/src/arrow/adapters/orc/CMakeLists.txt +++ b/cpp/src/arrow/adapters/orc/CMakeLists.txt @@ -49,7 +49,3 @@ add_arrow_test(adapter-test "orc" STATIC_LINK_LIBS ${ORC_STATIC_TEST_LINK_LIBS}) - -if(ARROW_ORC_JNI) - add_subdirectory(jni) -endif() \ No newline at end of file diff --git a/cpp/src/jni/CMakeLists.txt b/cpp/src/jni/CMakeLists.txt new file mode 100644 index 00000000000..d730ea19013 --- /dev/null +++ b/cpp/src/jni/CMakeLists.txt @@ -0,0 +1,24 @@ +# 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. + +# +# arrow_jni +# + +if(ARROW_ORC) + add_subdirectory(orc) +endif() diff --git a/cpp/src/arrow/adapters/orc/jni/CMakeLists.txt b/cpp/src/jni/orc/CMakeLists.txt similarity index 92% rename from cpp/src/arrow/adapters/orc/jni/CMakeLists.txt rename to cpp/src/jni/orc/CMakeLists.txt index 0088b94e93e..0ce4a6d0476 100644 --- a/cpp/src/arrow/adapters/orc/jni/CMakeLists.txt +++ b/cpp/src/jni/orc/CMakeLists.txt @@ -29,7 +29,7 @@ add_custom_target(arrow_orc_jni) set(JNI_HEADERS_DIR "${CMAKE_CURRENT_BINARY_DIR}/generated") -add_subdirectory(../../../../../../java/adapter/orc ./java) +add_subdirectory(../../../../java/adapter/orc ./java) set(ARROW_BUILD_STATIC OFF) @@ -52,4 +52,4 @@ ADD_ARROW_LIB(arrow_orc_jni OUTPUT_PATH ${CMAKE_CURRENT_BINARY_DIR}) -add_dependencies(arrow_orc_jni ${ARROW_ORC_JNI_LIBRARIES}) \ No newline at end of file +add_dependencies(arrow_orc_jni ${ARROW_ORC_JNI_LIBRARIES}) diff --git a/cpp/src/arrow/adapters/orc/jni/concurrent_map.h b/cpp/src/jni/orc/concurrent_map.h similarity index 100% rename from cpp/src/arrow/adapters/orc/jni/concurrent_map.h rename to cpp/src/jni/orc/concurrent_map.h diff --git a/cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp b/cpp/src/jni/orc/jni_wrapper.cpp similarity index 96% rename from cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp rename to cpp/src/jni/orc/jni_wrapper.cpp index 469340fa8d2..4df349d3563 100644 --- a/cpp/src/arrow/adapters/orc/jni/jni_wrapper.cpp +++ b/cpp/src/jni/orc/jni_wrapper.cpp @@ -49,7 +49,7 @@ static jint JNI_VERSION = JNI_VERSION_1_6; static arrow::concurrentMap> buffer_holder_; static arrow::concurrentMap> orc_stripe_reader_holder_; -static arrow::concurrentMap> orc_file_reader_holder_; +static arrow::concurrentMap> orc_reader_holder_; jclass CreateGlobalClassReference(JNIEnv* env, const char* class_name) { jclass local_class = env->FindClass(class_name); @@ -108,11 +108,11 @@ void JNI_OnUnload(JavaVM* vm, void* reserved) { buffer_holder_.Clear(); orc_stripe_reader_holder_.Clear(); - orc_file_reader_holder_.Clear(); + orc_reader_holder_.Clear(); } std::shared_ptr GetNativeReader(jlong id) { - return orc_file_reader_holder_.Lookup(id); + return orc_reader_holder_.Lookup(id); } std::shared_ptr GetStripeReader(jlong id) { @@ -167,7 +167,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_op env->ThrowNew(io_exception_class, std::string("Failed open file" + path).c_str()); } - return orc_file_reader_holder_.Insert( + return orc_reader_holder_.Insert( std::shared_ptr(reader.release())); } @@ -176,12 +176,12 @@ JNIEXPORT jlong JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_op JNIEXPORT void JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_close( JNIEnv* env, jclass this_cls, jlong id) { - orc_file_reader_holder_.Erase(id); + orc_reader_holder_.Erase(id); } JNIEXPORT jboolean JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_seek( JNIEnv* env, jclass this_cls, jlong id, jint row_number) { - auto reader = orc_file_reader_holder_.Lookup(id); + auto reader = orc_reader_holder_.Lookup(id); return reader->Seek(row_number).ok(); } @@ -189,7 +189,7 @@ JNIEXPORT jint JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_getNumberOfStripes(JNIEnv* env, jclass this_cls, jlong id) { - auto reader = orc_file_reader_holder_.Lookup(id); + auto reader = orc_reader_holder_.Lookup(id); return reader->NumberOfStripes(); } @@ -264,7 +264,7 @@ Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_next(JNIEnv* env, jobjectArray memory_array = env->NewObjectArray(buffers.size(), orc_memory_class, nullptr); - for (int j = 0; j < buffers.size(); ++j) { + for (size_t j = 0; j < buffers.size(); ++j) { auto buffer = buffers[j]; jobject memory = env->NewObject(orc_memory_class, orc_memory_constructor, buffer_holder_.Insert(buffer), buffer->data(), @@ -281,7 +281,7 @@ Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_next(JNIEnv* env, JNIEXPORT void JNICALL Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_close( JNIEnv* env, jclass this_cls, jlong id) { - buffer_holder_.Erase(id); + orc_stripe_reader_holder_.Erase(id); } JNIEXPORT void JNICALL Java_org_apache_arrow_adapter_orc_OrcMemoryJniWrapper_release( diff --git a/java/adapter/orc/pom.xml b/java/adapter/orc/pom.xml index 28f7b7d7d67..2a0c903a222 100644 --- a/java/adapter/orc/pom.xml +++ b/java/adapter/orc/pom.xml @@ -53,7 +53,7 @@ - ${arrow.cpp.build.dir}/src/arrow/adapters/orc/jni/ + ${arrow.cpp.build.dir}/src/jni/orc **/libarrow_orc_jni.* diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcFieldNode.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcFieldNode.java index aedbd61816b..716a1387660 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcFieldNode.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcFieldNode.java @@ -18,7 +18,7 @@ package org.apache.arrow.adapter.orc; /** - * Metadata about Vectors/Arrays that is passed via JNI interface + * Metadata about Vectors/Arrays that is passed via JNI interface. */ class OrcFieldNode { diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java index eb888f31f0a..dfbf26f1650 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java @@ -20,7 +20,7 @@ import java.io.IOException; /** - * JNI wrapper for Orc reader + * JNI wrapper for Orc reader. */ class OrcReaderJniWrapper { static { @@ -32,10 +32,10 @@ class OrcReaderJniWrapper { } /** - * Construct a orc file reader over the target file + * Construct a orc file reader over the target file. * @param fileName absolute file path of target file * @return id of the orc reader instance if file opened successfully, - * otherwise return error code * -1. + * otherwise return error code * -1. */ static native long open(String fileName); diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java index c21616a433a..3253f073047 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java @@ -31,7 +31,7 @@ class OrcRecordBatch { final List buffers; /** - * Construct a new instance + * Construct a new instance. * @param length number of records included in current batch * @param nodes meta data for each fields * @param buffers buffers for underlying data diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java index 476d23e7cd5..f7c276c778f 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java @@ -17,8 +17,6 @@ package org.apache.arrow.adapter.orc; -import io.netty.buffer.ArrowBuf; - import java.util.concurrent.atomic.AtomicInteger; import org.apache.arrow.memory.BufferAllocator; @@ -26,6 +24,8 @@ import org.apache.arrow.memory.ReferenceManager; import org.apache.arrow.util.Preconditions; +import io.netty.buffer.ArrowBuf; + /** * A simple reference manager implementation for memory allocated by native code. * The underlying memory will be released when reference count reach zero. diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java index 5594d488045..96900032843 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java @@ -17,8 +17,6 @@ package org.apache.arrow.adapter.orc; -import io.netty.buffer.ArrowBuf; - import java.io.IOException; import java.util.ArrayList; import java.util.stream.Collectors; @@ -35,7 +33,8 @@ import org.apache.arrow.vector.ipc.message.MessageSerializer; import org.apache.arrow.vector.types.pojo.Schema; import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel; -import sun.reflect.generics.reflectiveObjects.NotImplementedException; + +import io.netty.buffer.ArrowBuf; /** * Orc stripe that load data into ArrowRecordBatch. @@ -114,6 +113,6 @@ protected Schema readSchema() throws IOException { @Override protected ArrowDictionaryBatch readDictionary() throws IOException { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } } diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java index 02474b3234c..12896dc4791 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java @@ -43,7 +43,7 @@ class OrcStripeReaderJniWrapper { * Load next record batch. * @param readerId id of the stripe reader instance. * @return loaded record batch, return null when reached - * the end of current stripe. + * the end of current stripe. */ static native OrcRecordBatch next(long readerId); From e4c063041bac312a5f327b695ecbd754267847df Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Mon, 27 May 2019 18:05:12 +0800 Subject: [PATCH 07/21] remove redundant code --- cpp/src/jni/orc/jni_wrapper.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/cpp/src/jni/orc/jni_wrapper.cpp b/cpp/src/jni/orc/jni_wrapper.cpp index 4df349d3563..e7b67fbbd40 100644 --- a/cpp/src/jni/orc/jni_wrapper.cpp +++ b/cpp/src/jni/orc/jni_wrapper.cpp @@ -119,17 +119,6 @@ std::shared_ptr GetStripeReader(jlong id) { return orc_stripe_reader_holder_.Lookup(id); } -int jstr_to_cstr(JNIEnv* env, jstring jstr, char* cstr, size_t cstr_len) { - int32_t jlen, clen; - - clen = env->GetStringUTFLength(jstr); - if (clen > (int32_t)cstr_len) return -ENAMETOOLONG; - jlen = env->GetStringLength(jstr); - env->GetStringUTFRegion(jstr, 0, jlen, cstr); - if (env->ExceptionCheck()) return -EIO; - return 0; -} - std::string JStringToCString(JNIEnv* env, jstring string) { int32_t jlen, clen; clen = env->GetStringUTFLength(string); From 7a80fbd0d6fc11103766acd7fee59d703d88c9ae Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Tue, 28 May 2019 10:32:52 +0800 Subject: [PATCH 08/21] Minor refactor Convert OrcReaderJniWrapper to singleton model to allow client handle library loading exception --- cpp/src/jni/orc/jni_wrapper.cpp | 10 +++---- .../adapter/orc/OrcMemoryJniWrapper.java | 11 +------ .../apache/arrow/adapter/orc/OrcReader.java | 12 ++++---- .../adapter/orc/OrcReaderJniWrapper.java | 30 ++++++++++++------- .../arrow/adapter/orc/OrcStripeReader.java | 3 +- .../orc/OrcStripeReaderJniWrapper.java | 10 ------- 6 files changed, 34 insertions(+), 42 deletions(-) diff --git a/cpp/src/jni/orc/jni_wrapper.cpp b/cpp/src/jni/orc/jni_wrapper.cpp index e7b67fbbd40..403bba74f9e 100644 --- a/cpp/src/jni/orc/jni_wrapper.cpp +++ b/cpp/src/jni/orc/jni_wrapper.cpp @@ -133,7 +133,7 @@ extern "C" { #endif JNIEXPORT jlong JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_open( - JNIEnv* env, jclass this_cls, jstring file_path) { + JNIEnv* env, jobject this_obj, jstring file_path) { std::shared_ptr in_file; std::string path = JStringToCString(env, file_path); @@ -164,19 +164,19 @@ JNIEXPORT jlong JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_op } JNIEXPORT void JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_close( - JNIEnv* env, jclass this_cls, jlong id) { + JNIEnv* env, jobject this_obj, jlong id) { orc_reader_holder_.Erase(id); } JNIEXPORT jboolean JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_seek( - JNIEnv* env, jclass this_cls, jlong id, jint row_number) { + JNIEnv* env, jobject this_obj, jlong id, jint row_number) { auto reader = orc_reader_holder_.Lookup(id); return reader->Seek(row_number).ok(); } JNIEXPORT jint JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_getNumberOfStripes(JNIEnv* env, - jclass this_cls, + jobject this_obj, jlong id) { auto reader = orc_reader_holder_.Lookup(id); return reader->NumberOfStripes(); @@ -184,7 +184,7 @@ Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_getNumberOfStripes(JNIEnv* JNIEXPORT jlong JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_nextStripeReader(JNIEnv* env, - jclass this_cls, + jobject this_obj, jlong id, jlong batch_size) { auto reader = GetNativeReader(id); diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java index 6e3ab3634f5..6d602903a89 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java @@ -17,12 +17,11 @@ package org.apache.arrow.adapter.orc; -import java.io.IOException; - /** * Wrapper for orc memory allocated by native code. */ class OrcMemoryJniWrapper implements AutoCloseable { + private final long id; private final long memoryAddress; @@ -31,14 +30,6 @@ class OrcMemoryJniWrapper implements AutoCloseable { private final long capacity; - static { - try { - OrcJniUtils.loadOrcAdapterLibraryFromJar(); - } catch (IOException e) { - throw new ExceptionInInitializerError(e); - } - } - /** * Construct a new instance. * @param id unique id of the underlying memory. diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java index e0bd1a5a692..646da5625eb 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java @@ -29,6 +29,7 @@ * ArrowReader. */ public class OrcReader implements AutoCloseable { + private final OrcReaderJniWrapper jniWrapper; private BufferAllocator allocator; private final long id; @@ -41,7 +42,8 @@ public class OrcReader implements AutoCloseable { */ public OrcReader(String filePath, BufferAllocator allocator) throws IOException { this.allocator = allocator; - id = OrcReaderJniWrapper.open(filePath); + this.jniWrapper = OrcReaderJniWrapper.getInstance(); + this.id = jniWrapper.open(filePath); } /** @@ -51,7 +53,7 @@ public OrcReader(String filePath, BufferAllocator allocator) throws IOException * @return true if seek operation is succeeded */ public boolean seek(int rowNumber) { - return OrcReaderJniWrapper.seek(id, rowNumber); + return jniWrapper.seek(id, rowNumber); } /** @@ -61,7 +63,7 @@ public boolean seek(int rowNumber) { * @return ArrowReader that iterate over current stripes */ public ArrowReader nextStripeReader(long batchSize) { - long stripeReaderId = OrcReaderJniWrapper.nextStripeReader(id, batchSize); + long stripeReaderId = jniWrapper.nextStripeReader(id, batchSize); if (stripeReaderId < 0) { return null; } @@ -75,11 +77,11 @@ public ArrowReader nextStripeReader(long batchSize) { * @return number of stripes */ public int getNumberOfStripes() { - return OrcReaderJniWrapper.getNumberOfStripes(id); + return jniWrapper.getNumberOfStripes(id); } @Override public void close() throws Exception { - OrcReaderJniWrapper.close(id); + jniWrapper.close(id); } } diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java index dfbf26f1650..6c5e30a17d0 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java @@ -23,12 +23,20 @@ * JNI wrapper for Orc reader. */ class OrcReaderJniWrapper { - static { - try { - OrcJniUtils.loadOrcAdapterLibraryFromJar(); - } catch (IOException e) { - throw new ExceptionInInitializerError(e); + + private static volatile OrcReaderJniWrapper INSTANCE; + + static OrcReaderJniWrapper getInstance() throws IOException { + if (INSTANCE == null) { + synchronized (OrcReaderJniWrapper.class) { + if (INSTANCE == null) { + OrcJniUtils.loadOrcAdapterLibraryFromJar(); + INSTANCE = new OrcReaderJniWrapper(); + } + } } + + return INSTANCE; } /** @@ -37,13 +45,13 @@ class OrcReaderJniWrapper { * @return id of the orc reader instance if file opened successfully, * otherwise return error code * -1. */ - static native long open(String fileName); + native long open(String fileName); /** - * Close the underlying reader and release related resources. + * Release resources associated with designated reader instance. * @param readerId id of the reader instance. */ - static native void close(long readerId); + native void close(long readerId); /** * Seek to designated row. Invoke nextStripeReader() after seek @@ -52,14 +60,14 @@ class OrcReaderJniWrapper { * @param rowNumber the rows number to seek * @return true if seek operation is succeeded */ - static native boolean seek(long readerId, int rowNumber); + native boolean seek(long readerId, int rowNumber); /** * The number of stripes in the file. * @param readerId id of the reader instance * @return number of stripes */ - static native int getNumberOfStripes(long readerId); + native int getNumberOfStripes(long readerId); /** * Get a stripe level ArrowReader with specified batchSize in each record batch. @@ -67,5 +75,5 @@ class OrcReaderJniWrapper { * @param batchSize the number of rows loaded on each iteration * @return id of the stripe reader instance. */ - static native long nextStripeReader(long readerId, long batchSize); + native long nextStripeReader(long readerId, long batchSize); } diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java index 96900032843..92f73b0ce3a 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java @@ -45,7 +45,8 @@ public class OrcStripeReader extends ArrowReader { /** * Construct a new instance. - * @param id id of the stripe reader instance + * @param id id of the stripe reader instance, obtained by + * calling nextStripeReader from OrcReaderJniWrapper * @param allocator memory allocator for accounting. */ OrcStripeReader(long id, BufferAllocator allocator) { diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java index 12896dc4791..1dd96986108 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReaderJniWrapper.java @@ -17,21 +17,11 @@ package org.apache.arrow.adapter.orc; -import java.io.IOException; - /** * JNI wrapper for orc stripe reader. */ class OrcStripeReaderJniWrapper { - static { - try { - OrcJniUtils.loadOrcAdapterLibraryFromJar(); - } catch (IOException e) { - throw new ExceptionInInitializerError(e); - } - } - /** * Get the schema of current stripe. * @param readerId id of the stripe reader instance. From ce30933b1ac761e6dcede3a01b1b66e5286e6380 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Wed, 29 May 2019 16:25:55 +0800 Subject: [PATCH 09/21] Add Arrow Jni Reader Unittests --- cpp/src/jni/orc/jni_wrapper.cpp | 34 +++--- java/README.md | 4 +- java/adapter/orc/pom.xml | 52 ++++++++- .../adapter/orc/OrcReferenceManager.java | 3 +- .../arrow/adapter/orc/OrcReaderTest.java | 105 ++++++++++++++++++ java/pom.xml | 2 +- 6 files changed, 179 insertions(+), 21 deletions(-) create mode 100644 java/adapter/orc/src/test/java/org/apache/arrow/adapter/orc/OrcReaderTest.java diff --git a/cpp/src/jni/orc/jni_wrapper.cpp b/cpp/src/jni/orc/jni_wrapper.cpp index 403bba74f9e..fa7998308a9 100644 --- a/cpp/src/jni/orc/jni_wrapper.cpp +++ b/cpp/src/jni/orc/jni_wrapper.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include "org_apache_arrow_adapter_orc_OrcMemoryJniWrapper.h" #include "org_apache_arrow_adapter_orc_OrcReaderJniWrapper.h" @@ -75,22 +76,22 @@ jint JNI_OnLoad(JavaVM* vm, void* reserved) { return JNI_ERR; } - io_exception_class = CreateGlobalClassReference(env, "java/io/IOException"); - exception_class = CreateGlobalClassReference(env, "java/lang/Exception"); + io_exception_class = CreateGlobalClassReference(env, "Ljava/io/IOException;"); + exception_class = CreateGlobalClassReference(env, "Ljava/lang/Exception;"); orc_field_node_class = - CreateGlobalClassReference(env, "/org/apache/arrow/adapter/orc/OrcFieldNode"); + CreateGlobalClassReference(env, "Lorg/apache/arrow/adapter/orc/OrcFieldNode;"); orc_field_node_constructor = GetMethodID(env, orc_field_node_class, "", "(II)V"); orc_memory_class = CreateGlobalClassReference( - env, "/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper"); + env, "Lorg/apache/arrow/adapter/orc/OrcMemoryJniWrapper;"); orc_memory_constructor = GetMethodID(env, orc_memory_class, "", "(JJJJ)V"); record_batch_class = - CreateGlobalClassReference(env, "/org/apache/arrow/adapter/orc/OrcRecordBatch"); + CreateGlobalClassReference(env, "Lorg/apache/arrow/adapter/orc/OrcRecordBatch;"); record_batch_constructor = GetMethodID(env, record_batch_class, "", - "(I[L/org/apache/arrow/adapter/orc/OrcFieldNode;" - "[L/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper;)V"); + "(I[Lorg/apache/arrow/adapter/orc/OrcFieldNode;" + "[Lorg/apache/arrow/adapter/orc/OrcMemoryJniWrapper;)V"); env->ExceptionDescribe(); @@ -193,7 +194,10 @@ Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_nextStripeReader(JNIEnv* e if (!status.ok()) { return static_cast(status.code()) * -1; - ; + } + + if (!stripe_reader) { + return static_cast(arrow::StatusCode::Invalid) * -1; } return orc_stripe_reader_holder_.Insert(stripe_reader); @@ -214,7 +218,8 @@ Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_getSchema(JNIEnv* en } jbyteArray ret = env->NewByteArray(out->size()); - memcpy(env->GetByteArrayElements(ret, nullptr), out->data(), out->size()); + auto src = reinterpret_cast(out->data()); + env->SetByteArrayRegion(ret, 0, out->size(), src); return ret; } @@ -223,10 +228,9 @@ Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_next(JNIEnv* env, jclass this_cls, jlong id) { auto stripe_reader = GetStripeReader(id); - auto record_batch = new std::shared_ptr(); - auto status = stripe_reader->ReadNext(record_batch); - if (!status.ok()) { - delete record_batch; + std::shared_ptr record_batch; + auto status = stripe_reader->ReadNext(&record_batch); + if (!status.ok() || !record_batch) { return nullptr; } @@ -238,7 +242,7 @@ Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_next(JNIEnv* env, std::vector> buffers; for (int i = 0; i < schema->num_fields(); ++i) { - auto column = (*record_batch)->column(i); + auto column = record_batch->column(i); auto dataArray = column->data(); jobject field = env->NewObject(orc_field_node_class, orc_field_node_constructor, column->length(), column->null_count()); @@ -263,7 +267,7 @@ Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_next(JNIEnv* env, // create OrcRecordBatch jobject ret = env->NewObject(record_batch_class, record_batch_constructor, - (*record_batch)->num_rows(), field_array, memory_array); + record_batch->num_rows(), field_array, memory_array); return ret; } diff --git a/java/README.md b/java/README.md index df918a2bf98..cecdd400f37 100644 --- a/java/README.md +++ b/java/README.md @@ -45,13 +45,13 @@ mvn install -P gandiva -pl gandiva -am -Dgandiva.cpp.build.dir=../../debug This library is still in Alpha stages, and subject to API changes without deprecation warnings. -## Building and running tests for native orc (optional) +## Building and running tests for arrow jni (optional) Arrow Cpp must be built before this step. The cpp build directory must be provided as the value for argument arrow.cpp.build.dir. eg. ``` cd java -mvn install -P native-orc -pl adapter/orc -am -Darrow.cpp.build.dir=../../debug +mvn install -P native-orc -pl arrow-jni -am -Darrow.cpp.build.dir=../../release ``` ## Java Code Style Guide diff --git a/java/adapter/orc/pom.xml b/java/adapter/orc/pom.xml index 2a0c903a222..f718dd2d166 100644 --- a/java/adapter/orc/pom.xml +++ b/java/adapter/orc/pom.xml @@ -33,6 +33,56 @@ ${project.version} compile + + org.apache.orc + orc-core + 1.5.5 + test + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + commons-logging + commons-logging + + + + + org.apache.hadoop + hadoop-common + 2.2.0 + test + + + commons-logging + commons-logging + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + javax.servlet + servlet-api + + + + + org.apache.hive + hive-storage-api + 2.6.0 + test + @@ -47,7 +97,7 @@ Arrow Orc Adapter jar - ../../../cpp/debug-build/ + ../../../cpp/release-build/ diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java index f7c276c778f..69dcfdc553d 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java @@ -76,8 +76,7 @@ public void retain() { @Override public void retain(int increment) { Preconditions.checkArgument(increment > 0, "retain(%d) argument is not positive", increment); - final int originalReferenceCount = bufRefCnt.getAndAdd(increment); - Preconditions.checkArgument(originalReferenceCount > 0); + bufRefCnt.addAndGet(increment); } @Override diff --git a/java/adapter/orc/src/test/java/org/apache/arrow/adapter/orc/OrcReaderTest.java b/java/adapter/orc/src/test/java/org/apache/arrow/adapter/orc/OrcReaderTest.java new file mode 100644 index 00000000000..943b2cb230f --- /dev/null +++ b/java/adapter/orc/src/test/java/org/apache/arrow/adapter/orc/OrcReaderTest.java @@ -0,0 +1,105 @@ +/* + * 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.arrow.adapter.orc; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; + +import java.io.File; +import java.nio.charset.StandardCharsets; +import java.util.List; + + +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.ipc.ArrowReader; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.orc.OrcFile; +import org.apache.orc.TypeDescription; +import org.apache.orc.Writer; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + + + +public class OrcReaderTest { + + @Rule + public TemporaryFolder testFolder = new TemporaryFolder(); + + private static final int MAX_ALLOCATION = 8 * 1024; + private static RootAllocator allocator; + + @BeforeClass + public static void beforeClass() { + allocator = new RootAllocator(MAX_ALLOCATION); + } + + @Test + public void testOrcJniReader() throws Exception { + TypeDescription schema = TypeDescription.fromString("struct"); + File testFile = new File(testFolder.getRoot(), "test-orc"); + + Writer writer = OrcFile.createWriter(new Path(testFile.getAbsolutePath()), + OrcFile.writerOptions(new Configuration()).setSchema(schema)); + VectorizedRowBatch batch = schema.createRowBatch(); + LongColumnVector longColumnVector = (LongColumnVector) batch.cols[0]; + BytesColumnVector bytesColumnVector = (BytesColumnVector) batch.cols[1]; + for (int r = 0; r < 1024; ++r) { + int row = batch.size++; + longColumnVector.vector[row] = r; + byte[] buffer = ("Last-" + (r * 3)).getBytes(StandardCharsets.UTF_8); + bytesColumnVector.setRef(row, buffer, 0, buffer.length); + } + writer.addRowBatch(batch); + writer.close(); + + OrcReader reader = new OrcReader(testFile.getAbsolutePath(), allocator); + assertEquals(1, reader.getNumberOfStripes()); + + ArrowReader stripeReader = reader.nextStripeReader(1024); + VectorSchemaRoot schemaRoot = stripeReader.getVectorSchemaRoot(); + stripeReader.loadNextBatch(); + + List fields = schemaRoot.getFieldVectors(); + assertEquals(2, fields.size()); + + IntVector intVector = (IntVector)fields.get(0); + VarCharVector varCharVector = (VarCharVector)fields.get(1); + for (int i = 0; i < 1024; ++i) { + assertEquals(i, intVector.get(i)); + assertEquals("Last-" + (i * 3), new String(varCharVector.get(i), StandardCharsets.UTF_8)); + } + + assertFalse(stripeReader.loadNextBatch()); + assertNull(reader.nextStripeReader(1024)); + + stripeReader.close(); + reader.close(); + } +} diff --git a/java/pom.xml b/java/pom.xml index dcea56302f6..0cdb2a32086 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -685,7 +685,7 @@ - native-orc + arrow-jni format memory From 26d74db82c0d21fe411fd4d13244492deba6b33c Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Wed, 29 May 2019 16:36:58 +0800 Subject: [PATCH 10/21] fix minor style check error --- .../main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java index 3253f073047..a006cacab98 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcRecordBatch.java @@ -20,6 +20,9 @@ import java.util.Arrays; import java.util.List; +/** + * Wrapper for record batch meta and native memory. + */ class OrcRecordBatch { final int length; From 4f89e345419f9a929b4b2935bac5bc5d477c7060 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Wed, 29 May 2019 16:42:23 +0800 Subject: [PATCH 11/21] Make sure resources are properly released. --- .../java/org/apache/arrow/adapter/orc/OrcStripeReader.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java index 92f73b0ce3a..8406892f2fc 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java @@ -88,8 +88,11 @@ public long bytesRead() { @Override protected void closeReadSource() throws IOException { - OrcStripeReaderJniWrapper.close(id); - schemaReader.close(); + try { + OrcStripeReaderJniWrapper.close(id); + } finally { + schemaReader.close(); + } } @Override From f2a0c04b889b430d366ce2b5a45fb8278ae24c63 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Wed, 29 May 2019 16:49:21 +0800 Subject: [PATCH 12/21] destruct schema reader when finish reading --- .../org/apache/arrow/adapter/orc/OrcStripeReader.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java index 8406892f2fc..0eb085a265c 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java @@ -88,11 +88,7 @@ public long bytesRead() { @Override protected void closeReadSource() throws IOException { - try { - OrcStripeReaderJniWrapper.close(id); - } finally { - schemaReader.close(); - } + OrcStripeReaderJniWrapper.close(id); } @Override @@ -112,6 +108,8 @@ protected Schema readSchema() throws IOException { throw new IOException("Expected schema but header was " + result.getMessage().headerType()); } + schemaReader.close(); + return MessageSerializer.deserializeSchema(result.getMessage()); } From 44505df5dbfa4d283475304a4649189ceaf441f1 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Wed, 29 May 2019 17:46:51 +0800 Subject: [PATCH 13/21] Fix cmake format --- cpp/cmake_modules/BuildUtils.cmake | 3 +- cpp/src/jni/CMakeLists.txt | 2 +- cpp/src/jni/orc/CMakeLists.txt | 36 +++++++++---------- .../arrow/adapter/orc/OrcStripeReader.java | 3 +- 4 files changed, 21 insertions(+), 23 deletions(-) diff --git a/cpp/cmake_modules/BuildUtils.cmake b/cpp/cmake_modules/BuildUtils.cmake index 048ef66e299..5f042548885 100644 --- a/cpp/cmake_modules/BuildUtils.cmake +++ b/cpp/cmake_modules/BuildUtils.cmake @@ -319,8 +319,7 @@ function(ADD_ARROW_LIB LIB_NAME) endif() set_target_properties(${LIB_NAME}_static - PROPERTIES LIBRARY_OUTPUT_DIRECTORY - "${OUTPUT_PATH}" OUTPUT_NAME + PROPERTIES LIBRARY_OUTPUT_DIRECTORY "${OUTPUT_PATH}" OUTPUT_NAME ${LIB_NAME_STATIC}) if(ARG_STATIC_INSTALL_INTERFACE_LIBS) diff --git a/cpp/src/jni/CMakeLists.txt b/cpp/src/jni/CMakeLists.txt index d730ea19013..3872d671934 100644 --- a/cpp/src/jni/CMakeLists.txt +++ b/cpp/src/jni/CMakeLists.txt @@ -20,5 +20,5 @@ # if(ARROW_ORC) - add_subdirectory(orc) + add_subdirectory(orc) endif() diff --git a/cpp/src/jni/orc/CMakeLists.txt b/cpp/src/jni/orc/CMakeLists.txt index 0ce4a6d0476..54705fcfc48 100644 --- a/cpp/src/jni/orc/CMakeLists.txt +++ b/cpp/src/jni/orc/CMakeLists.txt @@ -33,23 +33,23 @@ add_subdirectory(../../../../java/adapter/orc ./java) set(ARROW_BUILD_STATIC OFF) -ADD_ARROW_LIB(arrow_orc_jni - BUILD_SHARED - SOURCES - jni_wrapper.cpp - OUTPUTS - ARROW_ORC_JNI_LIBRARIES - SHARED_PRIVATE_LINK_LIBS - arrow_static - EXTRA_INCLUDES - ${JNI_HEADERS_DIR} - PRIVATE_INCLUDES - ${JNI_INCLUDE_DIRS} - ${CMAKE_CURRENT_BINARY_DIR} - DEPENDENCIES - arrow_static - arrow_orc_java - OUTPUT_PATH - ${CMAKE_CURRENT_BINARY_DIR}) +add_arrow_lib(arrow_orc_jni + BUILD_SHARED + SOURCES + jni_wrapper.cpp + OUTPUTS + ARROW_ORC_JNI_LIBRARIES + SHARED_PRIVATE_LINK_LIBS + arrow_static + EXTRA_INCLUDES + ${JNI_HEADERS_DIR} + PRIVATE_INCLUDES + ${JNI_INCLUDE_DIRS} + ${CMAKE_CURRENT_BINARY_DIR} + DEPENDENCIES + arrow_static + arrow_orc_java + OUTPUT_PATH + ${CMAKE_CURRENT_BINARY_DIR}) add_dependencies(arrow_orc_jni ${ARROW_ORC_JNI_LIBRARIES}) diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java index 0eb085a265c..98c12945eda 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java @@ -99,6 +99,7 @@ protected Schema readSchema() throws IOException { new ByteArrayReadableSeekableByteChannel(schemaBytes)), allocator); MessageResult result = schemaReader.readNext(); + schemaReader.close(); if (result == null) { throw new IOException("Unexpected end of input. Missing schema."); @@ -108,8 +109,6 @@ protected Schema readSchema() throws IOException { throw new IOException("Expected schema but header was " + result.getMessage().headerType()); } - schemaReader.close(); - return MessageSerializer.deserializeSchema(result.getMessage()); } From dd981af5fb5a521a6da7c7da6bb843de6cc1b535 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Wed, 29 May 2019 18:12:52 +0800 Subject: [PATCH 14/21] fix lint and clang-format --- cpp/src/jni/orc/concurrent_map.h | 85 ++++++++++++++++---------------- cpp/src/jni/orc/jni_wrapper.cpp | 14 +++--- 2 files changed, 50 insertions(+), 49 deletions(-) diff --git a/cpp/src/jni/orc/concurrent_map.h b/cpp/src/jni/orc/concurrent_map.h index 8763a6db6ca..5ae30d2cced 100644 --- a/cpp/src/jni/orc/concurrent_map.h +++ b/cpp/src/jni/orc/concurrent_map.h @@ -19,62 +19,63 @@ #define JNI_ID_TO_MODULE_MAP_H #include +#include #include namespace arrow { template class concurrentMap { -public: - concurrentMap() : module_id_(kInitModuleId) {} + public: + concurrentMap() : module_id_(kInitModuleId) {} - jlong Insert(HOLDER holder) { - mtx_.lock(); - jlong result = module_id_++; - map_.insert(std::pair(result, holder)); - mtx_.unlock(); - return result; - } + jlong Insert(HOLDER holder) { + mtx_.lock(); + jlong result = module_id_++; + map_.insert(std::pair(result, holder)); + mtx_.unlock(); + return result; + } - void Erase(jlong module_id) { - mtx_.lock(); - map_.erase(module_id); - mtx_.unlock(); - } + void Erase(jlong module_id) { + mtx_.lock(); + map_.erase(module_id); + mtx_.unlock(); + } - HOLDER Lookup(jlong module_id) { - HOLDER result = nullptr; - try { - result = map_.at(module_id); - } catch (const std::out_of_range &e) { - } - if (result != nullptr) { - return result; - } - mtx_.lock(); - try { - result = map_.at(module_id); - } catch (const std::out_of_range &e) { - } - mtx_.unlock(); - return result; + HOLDER Lookup(jlong module_id) { + HOLDER result = nullptr; + try { + result = map_.at(module_id); + } catch (const std::out_of_range& e) { } - - void Clear() { - mtx_.lock(); - map_.clear(); - mtx_.unlock(); + if (result != nullptr) { + return result; } + mtx_.lock(); + try { + result = map_.at(module_id); + } catch (const std::out_of_range& e) { + } + mtx_.unlock(); + return result; + } + + void Clear() { + mtx_.lock(); + map_.clear(); + mtx_.unlock(); + } -private: - static const int kInitModuleId = 4; + private: + static const int kInitModuleId = 4; - long module_id_; - std::mutex mtx_; - // map from module ids returned to Java and module pointers - std::unordered_map map_; + int64_t module_id_; + std::mutex mtx_; + // map from module ids returned to Java and module pointers + std::unordered_map map_; }; -} // namespace gandiva +} // namespace arrow #endif // JNI_ID_TO_MODULE_MAP_H diff --git a/cpp/src/jni/orc/jni_wrapper.cpp b/cpp/src/jni/orc/jni_wrapper.cpp index fa7998308a9..67680037ed5 100644 --- a/cpp/src/jni/orc/jni_wrapper.cpp +++ b/cpp/src/jni/orc/jni_wrapper.cpp @@ -22,14 +22,14 @@ #include #include #include -#include #include +#include #include "org_apache_arrow_adapter_orc_OrcMemoryJniWrapper.h" #include "org_apache_arrow_adapter_orc_OrcReaderJniWrapper.h" #include "org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper.h" -#include "concurrent_map.h" +#include "./concurrent_map.h" using ORCFileReader = arrow::adapters::orc::ORCFileReader; using RecordBatchReader = arrow::RecordBatchReader; @@ -89,9 +89,10 @@ jint JNI_OnLoad(JavaVM* vm, void* reserved) { record_batch_class = CreateGlobalClassReference(env, "Lorg/apache/arrow/adapter/orc/OrcRecordBatch;"); - record_batch_constructor = GetMethodID(env, record_batch_class, "", - "(I[Lorg/apache/arrow/adapter/orc/OrcFieldNode;" - "[Lorg/apache/arrow/adapter/orc/OrcMemoryJniWrapper;)V"); + record_batch_constructor = + GetMethodID(env, record_batch_class, "", + "(I[Lorg/apache/arrow/adapter/orc/OrcFieldNode;" + "[Lorg/apache/arrow/adapter/orc/OrcMemoryJniWrapper;)V"); env->ExceptionDescribe(); @@ -157,8 +158,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_op env->ThrowNew(io_exception_class, std::string("Failed open file" + path).c_str()); } - return orc_reader_holder_.Insert( - std::shared_ptr(reader.release())); + return orc_reader_holder_.Insert(std::shared_ptr(reader.release())); } return static_cast(ret.code()) * -1; From 9b13d7f62338fa53e39d3d7e892f6ac3347163f3 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Thu, 30 May 2019 10:34:47 +0800 Subject: [PATCH 15/21] replace nullptr with NULLPTR macro --- cpp/src/jni/orc/concurrent_map.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/cpp/src/jni/orc/concurrent_map.h b/cpp/src/jni/orc/concurrent_map.h index 5ae30d2cced..872721ae603 100644 --- a/cpp/src/jni/orc/concurrent_map.h +++ b/cpp/src/jni/orc/concurrent_map.h @@ -22,6 +22,8 @@ #include #include +#include "arrow/util/macros.h" + namespace arrow { template @@ -44,12 +46,12 @@ class concurrentMap { } HOLDER Lookup(jlong module_id) { - HOLDER result = nullptr; + HOLDER result = NULLPTR; try { result = map_.at(module_id); } catch (const std::out_of_range& e) { } - if (result != nullptr) { + if (result != NULLPTR) { return result; } mtx_.lock(); From 9b04b762d64ed39f6d55798693bc309a83c05117 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Fri, 31 May 2019 14:22:38 +0800 Subject: [PATCH 16/21] fix style issues and add proper docs --- cpp/build-support/lint_cpp_cli.py | 1 + cpp/src/jni/orc/concurrent_map.h | 34 +++++++------ cpp/src/jni/orc/jni_wrapper.cpp | 48 ++++++++----------- .../apache/arrow/adapter/orc/OrcJniUtils.java | 17 ++----- .../adapter/orc/OrcMemoryJniWrapper.java | 12 +++++ 5 files changed, 58 insertions(+), 54 deletions(-) diff --git a/cpp/build-support/lint_cpp_cli.py b/cpp/build-support/lint_cpp_cli.py index ab2de5901a4..e0fee00cafa 100644 --- a/cpp/build-support/lint_cpp_cli.py +++ b/cpp/build-support/lint_cpp_cli.py @@ -77,6 +77,7 @@ def lint_file(path): arrow/visitor_inline.h gandiva/cache.h gandiva/jni + jni/ test internal''') diff --git a/cpp/src/jni/orc/concurrent_map.h b/cpp/src/jni/orc/concurrent_map.h index 872721ae603..2eabbfdde0d 100644 --- a/cpp/src/jni/orc/concurrent_map.h +++ b/cpp/src/jni/orc/concurrent_map.h @@ -19,34 +19,38 @@ #define JNI_ID_TO_MODULE_MAP_H #include +#include #include #include #include "arrow/util/macros.h" namespace arrow { +namespace jni { -template +/** + * An utility class that map module id to module pointers. + * @tparam Holder class of the object to hold. + */ +template class concurrentMap { public: concurrentMap() : module_id_(kInitModuleId) {} - jlong Insert(HOLDER holder) { - mtx_.lock(); + jlong Insert(Holder holder) { + std::lock_guard lock(mtx_); jlong result = module_id_++; - map_.insert(std::pair(result, holder)); - mtx_.unlock(); + map_.insert(std::pair(result, holder)); return result; } void Erase(jlong module_id) { - mtx_.lock(); + std::lock_guard lock(mtx_); map_.erase(module_id); - mtx_.unlock(); } - HOLDER Lookup(jlong module_id) { - HOLDER result = NULLPTR; + Holder Lookup(jlong module_id) { + Holder result = NULLPTR; try { result = map_.at(module_id); } catch (const std::out_of_range& e) { @@ -54,30 +58,30 @@ class concurrentMap { if (result != NULLPTR) { return result; } - mtx_.lock(); + std::lock_guard lock(mtx_); try { result = map_.at(module_id); } catch (const std::out_of_range& e) { } - mtx_.unlock(); return result; } void Clear() { - mtx_.lock(); + std::lock_guard lock(mtx_); map_.clear(); - mtx_.unlock(); } private: - static const int kInitModuleId = 4; + // starting value of the module_id. + static constexpr int kInitModuleId = 4; int64_t module_id_; std::mutex mtx_; // map from module ids returned to Java and module pointers - std::unordered_map map_; + std::unordered_map map_; }; +} // namespace jni } // namespace arrow #endif // JNI_ID_TO_MODULE_MAP_H diff --git a/cpp/src/jni/orc/jni_wrapper.cpp b/cpp/src/jni/orc/jni_wrapper.cpp index 67680037ed5..6982dfdc9b5 100644 --- a/cpp/src/jni/orc/jni_wrapper.cpp +++ b/cpp/src/jni/orc/jni_wrapper.cpp @@ -48,9 +48,11 @@ static jmethodID record_batch_constructor; static jint JNI_VERSION = JNI_VERSION_1_6; -static arrow::concurrentMap> buffer_holder_; -static arrow::concurrentMap> orc_stripe_reader_holder_; -static arrow::concurrentMap> orc_reader_holder_; +using arrow::jni::concurrentMap; + +static concurrentMap> buffer_holder_; +static concurrentMap> orc_stripe_reader_holder_; +static concurrentMap> orc_reader_holder_; jclass CreateGlobalClassReference(JNIEnv* env, const char* class_name) { jclass local_class = env->FindClass(class_name); @@ -70,6 +72,19 @@ jmethodID GetMethodID(JNIEnv* env, jclass this_class, const char* name, const ch return ret; } +std::string JStringToCString(JNIEnv* env, jstring string) { + int32_t jlen, clen; + clen = env->GetStringUTFLength(string); + jlen = env->GetStringLength(string); + std::vector buffer(clen); + env->GetStringUTFRegion(string, 0, jlen, buffer.data()); + return std::string(buffer.data(), clen); +} + +#ifdef __cplusplus +extern "C" { +#endif + jint JNI_OnLoad(JavaVM* vm, void* reserved) { JNIEnv* env; if (vm->GetEnv(reinterpret_cast(&env), JNI_VERSION) != JNI_OK) { @@ -113,27 +128,6 @@ void JNI_OnUnload(JavaVM* vm, void* reserved) { orc_reader_holder_.Clear(); } -std::shared_ptr GetNativeReader(jlong id) { - return orc_reader_holder_.Lookup(id); -} - -std::shared_ptr GetStripeReader(jlong id) { - return orc_stripe_reader_holder_.Lookup(id); -} - -std::string JStringToCString(JNIEnv* env, jstring string) { - int32_t jlen, clen; - clen = env->GetStringUTFLength(string); - jlen = env->GetStringLength(string); - std::vector buffer(clen); - env->GetStringUTFRegion(string, 0, jlen, buffer.data()); - return std::string(buffer.data(), clen); -} - -#ifdef __cplusplus -extern "C" { -#endif - JNIEXPORT jlong JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_open( JNIEnv* env, jobject this_obj, jstring file_path) { std::shared_ptr in_file; @@ -188,7 +182,7 @@ Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_nextStripeReader(JNIEnv* e jobject this_obj, jlong id, jlong batch_size) { - auto reader = GetNativeReader(id); + auto reader = orc_reader_holder_.Lookup(id); std::shared_ptr stripe_reader; auto status = reader->NextStripeReader(batch_size, &stripe_reader); @@ -207,7 +201,7 @@ JNIEXPORT jbyteArray JNICALL Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_getSchema(JNIEnv* env, jclass this_cls, jlong id) { - auto stripe_reader = GetStripeReader(id); + auto stripe_reader = orc_stripe_reader_holder_.Lookup(id); auto schema = stripe_reader->schema(); std::shared_ptr out; @@ -227,7 +221,7 @@ JNIEXPORT jobject JNICALL Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_next(JNIEnv* env, jclass this_cls, jlong id) { - auto stripe_reader = GetStripeReader(id); + auto stripe_reader = orc_stripe_reader_holder_.Lookup(id); std::shared_ptr record_batch; auto status = stripe_reader->ReadNext(&record_batch); if (!status.ok() || !record_batch) { diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java index c14cb7454e2..a18df9b966d 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java @@ -18,10 +18,10 @@ package org.apache.arrow.adapter.orc; import java.io.File; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.nio.file.Files; -import java.nio.file.InvalidPathException; import java.nio.file.StandardCopyOption; import java.util.UUID; @@ -32,6 +32,8 @@ class OrcJniUtils { private static final String LIBRARY_NAME = "arrow_orc_jni"; private static boolean isLoaded = false; + private OrcJniUtils() {} + /** * Load arrow orc jni library from jar. * @throws IOException throws IOException in case target library can not be found. @@ -55,7 +57,7 @@ private static File moveFileFromJarToTemp(final String tmpDir, String libraryToL try (final InputStream is = OrcReaderJniWrapper.class.getClassLoader() .getResourceAsStream(libraryToLoad)) { if (is == null) { - throw new InvalidPathException(libraryToLoad, "file was not found inside JAR."); + throw new FileNotFoundException(libraryToLoad); } else { Files.copy(is, temp.toPath(), StandardCopyOption.REPLACE_EXISTING); } @@ -66,15 +68,6 @@ private static File moveFileFromJarToTemp(final String tmpDir, String libraryToL private static File setupFile(String tmpDir, String libraryToLoad) throws IOException { final String randomizeFileName = libraryToLoad + UUID.randomUUID(); - final File temp = new File(tmpDir, randomizeFileName); - if (temp.exists() && !temp.delete()) { - throw new InvalidPathException( - temp.getAbsolutePath(), "File already exists and cannot be removed."); - } - if (!temp.createNewFile()) { - throw new InvalidPathException(temp.getAbsolutePath(), "File could not be created."); - } - temp.deleteOnExit(); - return temp; + return File.createTempFile(tmpDir, randomizeFileName); } } diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java index 6d602903a89..419271821d2 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java @@ -44,14 +44,26 @@ class OrcMemoryJniWrapper implements AutoCloseable { this.capacity = capacity; } + /** + * Return the size of underlying chunk of memory that has valid data. + * @return + */ long getSize() { return size; } + /** + * Return the size of underlying chunk of memory managed by this OrcMemoryJniWrapper. + * @return + */ long getCapacity() { return capacity; } + /** + * Return the memory address of underlying chunk of memory. + * @return + */ long getMemoryAddress() { return memoryAddress; } From fc801756d28b6caad47462b236c5bf06570b05f8 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Fri, 31 May 2019 17:42:47 +0800 Subject: [PATCH 17/21] resolve comments --- cpp/src/jni/orc/concurrent_map.h | 24 ++++++++----------- cpp/src/jni/orc/jni_wrapper.cpp | 8 +++---- .../apache/arrow/adapter/orc/OrcJniUtils.java | 13 +--------- .../adapter/orc/OrcMemoryJniWrapper.java | 6 ++--- .../apache/arrow/adapter/orc/OrcReader.java | 3 +++ .../arrow/adapter/orc/OrcStripeReader.java | 3 +++ 6 files changed, 24 insertions(+), 33 deletions(-) diff --git a/cpp/src/jni/orc/concurrent_map.h b/cpp/src/jni/orc/concurrent_map.h index 2eabbfdde0d..cc875030b7c 100644 --- a/cpp/src/jni/orc/concurrent_map.h +++ b/cpp/src/jni/orc/concurrent_map.h @@ -33,9 +33,9 @@ namespace jni { * @tparam Holder class of the object to hold. */ template -class concurrentMap { +class ConcurrentMap { public: - concurrentMap() : module_id_(kInitModuleId) {} + ConcurrentMap() : module_id_(init_module_id_) {} jlong Insert(Holder holder) { std::lock_guard lock(mtx_); @@ -50,20 +50,16 @@ class concurrentMap { } Holder Lookup(jlong module_id) { - Holder result = NULLPTR; - try { - result = map_.at(module_id); - } catch (const std::out_of_range& e) { - } - if (result != NULLPTR) { - return result; + auto it = map_.find(module_id); + if (it != map_.end()) { + return it->second; } std::lock_guard lock(mtx_); - try { - result = map_.at(module_id); - } catch (const std::out_of_range& e) { + it = map_.find(module_id); + if (it != map_.end()) { + return it->second; } - return result; + return NULLPTR; } void Clear() { @@ -73,7 +69,7 @@ class concurrentMap { private: // starting value of the module_id. - static constexpr int kInitModuleId = 4; + static constexpr int init_module_id_ = 4; int64_t module_id_; std::mutex mtx_; diff --git a/cpp/src/jni/orc/jni_wrapper.cpp b/cpp/src/jni/orc/jni_wrapper.cpp index 6982dfdc9b5..38e4c80205d 100644 --- a/cpp/src/jni/orc/jni_wrapper.cpp +++ b/cpp/src/jni/orc/jni_wrapper.cpp @@ -48,11 +48,11 @@ static jmethodID record_batch_constructor; static jint JNI_VERSION = JNI_VERSION_1_6; -using arrow::jni::concurrentMap; +using arrow::jni::ConcurrentMap; -static concurrentMap> buffer_holder_; -static concurrentMap> orc_stripe_reader_holder_; -static concurrentMap> orc_reader_holder_; +static ConcurrentMap> buffer_holder_; +static ConcurrentMap> orc_stripe_reader_holder_; +static ConcurrentMap> orc_reader_holder_; jclass CreateGlobalClassReference(JNIEnv* env, const char* class_name) { jclass local_class = env->FindClass(class_name); diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java index a18df9b966d..721827bc91a 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java @@ -23,7 +23,6 @@ import java.io.InputStream; import java.nio.file.Files; import java.nio.file.StandardCopyOption; -import java.util.UUID; /** * Helper class for JNI related operations. @@ -34,10 +33,6 @@ class OrcJniUtils { private OrcJniUtils() {} - /** - * Load arrow orc jni library from jar. - * @throws IOException throws IOException in case target library can not be found. - */ static void loadOrcAdapterLibraryFromJar() throws IOException { synchronized (OrcJniUtils.class) { @@ -53,7 +48,7 @@ static void loadOrcAdapterLibraryFromJar() private static File moveFileFromJarToTemp(final String tmpDir, String libraryToLoad) throws IOException { - final File temp = setupFile(tmpDir, libraryToLoad); + final File temp = File.createTempFile(tmpDir, libraryToLoad); try (final InputStream is = OrcReaderJniWrapper.class.getClassLoader() .getResourceAsStream(libraryToLoad)) { if (is == null) { @@ -64,10 +59,4 @@ private static File moveFileFromJarToTemp(final String tmpDir, String libraryToL } return temp; } - - private static File setupFile(String tmpDir, String libraryToLoad) - throws IOException { - final String randomizeFileName = libraryToLoad + UUID.randomUUID(); - return File.createTempFile(tmpDir, randomizeFileName); - } } diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java index 419271821d2..e1e886f77f8 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java @@ -46,7 +46,7 @@ class OrcMemoryJniWrapper implements AutoCloseable { /** * Return the size of underlying chunk of memory that has valid data. - * @return + * @return valid data size */ long getSize() { return size; @@ -54,7 +54,7 @@ long getSize() { /** * Return the size of underlying chunk of memory managed by this OrcMemoryJniWrapper. - * @return + * @return underlying memory size */ long getCapacity() { return capacity; @@ -62,7 +62,7 @@ long getCapacity() { /** * Return the memory address of underlying chunk of memory. - * @return + * @return memory address */ long getMemoryAddress() { return memoryAddress; diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java index 646da5625eb..c418ed10ba1 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java @@ -32,6 +32,9 @@ public class OrcReader implements AutoCloseable { private final OrcReaderJniWrapper jniWrapper; private BufferAllocator allocator; + /** + * reference to native reader instance. + */ private final long id; /** diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java index 98c12945eda..1f1288eff6f 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java @@ -40,6 +40,9 @@ * Orc stripe that load data into ArrowRecordBatch. */ public class OrcStripeReader extends ArrowReader { + /** + * reference to native stripe reader instance. + */ private final long id; private MessageChannelReader schemaReader; From de8529cf1874103edad5f926bdc9daa082ef8d86 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Tue, 4 Jun 2019 16:36:13 +0800 Subject: [PATCH 18/21] resolve comments --- cpp/src/jni/orc/jni_wrapper.cpp | 17 +++++++- .../adapter/orc/OrcMemoryJniWrapper.java | 10 ++--- .../apache/arrow/adapter/orc/OrcReader.java | 12 +++--- .../adapter/orc/OrcReferenceManager.java | 13 +----- .../arrow/adapter/orc/OrcStripeReader.java | 40 +++++++++---------- 5 files changed, 48 insertions(+), 44 deletions(-) diff --git a/cpp/src/jni/orc/jni_wrapper.cpp b/cpp/src/jni/orc/jni_wrapper.cpp index 38e4c80205d..56eff805b21 100644 --- a/cpp/src/jni/orc/jni_wrapper.cpp +++ b/cpp/src/jni/orc/jni_wrapper.cpp @@ -136,7 +136,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_op arrow::Status ret; if (path.find("hdfs://") == 0) { - env->ThrowNew(io_exception_class, "hdfs path not support yet."); + env->ThrowNew(io_exception_class, "hdfs path not supported yet."); } else { ret = arrow::io::ReadableFile::Open(path, &in_file); } @@ -183,6 +183,11 @@ Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_nextStripeReader(JNIEnv* e jlong id, jlong batch_size) { auto reader = orc_reader_holder_.Lookup(id); + if (!reader) { + std::string error_message = "invalid reader id " + std::to_string(id); + env->ThrowNew(exception_class, error_message.c_str()); + } + std::shared_ptr stripe_reader; auto status = reader->NextStripeReader(batch_size, &stripe_reader); @@ -202,6 +207,11 @@ Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_getSchema(JNIEnv* en jclass this_cls, jlong id) { auto stripe_reader = orc_stripe_reader_holder_.Lookup(id); + if (!stripe_reader) { + std::string error_message = "invalid stripe reader id " + std::to_string(id); + env->ThrowNew(exception_class, error_message.c_str()); + } + auto schema = stripe_reader->schema(); std::shared_ptr out; @@ -222,6 +232,11 @@ Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_next(JNIEnv* env, jclass this_cls, jlong id) { auto stripe_reader = orc_stripe_reader_holder_.Lookup(id); + if (!stripe_reader) { + std::string error_message = "invalid stripe reader id " + std::to_string(id); + env->ThrowNew(exception_class, error_message.c_str()); + } + std::shared_ptr record_batch; auto status = stripe_reader->ReadNext(&record_batch); if (!status.ok() || !record_batch) { diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java index e1e886f77f8..27f54b75c1c 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcMemoryJniWrapper.java @@ -22,7 +22,7 @@ */ class OrcMemoryJniWrapper implements AutoCloseable { - private final long id; + private final long nativeInstanceId; private final long memoryAddress; @@ -32,13 +32,13 @@ class OrcMemoryJniWrapper implements AutoCloseable { /** * Construct a new instance. - * @param id unique id of the underlying memory. + * @param nativeInstanceId unique id of the underlying memory. * @param memoryAddress starting memory address of the the underlying memory. * @param size size of the valid data. * @param capacity allocated memory size. */ - OrcMemoryJniWrapper(long id, long memoryAddress, long size, long capacity) { - this.id = id; + OrcMemoryJniWrapper(long nativeInstanceId, long memoryAddress, long size, long capacity) { + this.nativeInstanceId = nativeInstanceId; this.memoryAddress = memoryAddress; this.size = size; this.capacity = capacity; @@ -70,7 +70,7 @@ long getMemoryAddress() { @Override public void close() { - release(id); + release(nativeInstanceId); } private native void release(long id); diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java index c418ed10ba1..7f95f65f775 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java @@ -35,7 +35,7 @@ public class OrcReader implements AutoCloseable { /** * reference to native reader instance. */ - private final long id; + private final long nativeInstanceId; /** * Create an OrcReader that iterate over orc stripes. @@ -46,7 +46,7 @@ public class OrcReader implements AutoCloseable { public OrcReader(String filePath, BufferAllocator allocator) throws IOException { this.allocator = allocator; this.jniWrapper = OrcReaderJniWrapper.getInstance(); - this.id = jniWrapper.open(filePath); + this.nativeInstanceId = jniWrapper.open(filePath); } /** @@ -56,7 +56,7 @@ public OrcReader(String filePath, BufferAllocator allocator) throws IOException * @return true if seek operation is succeeded */ public boolean seek(int rowNumber) { - return jniWrapper.seek(id, rowNumber); + return jniWrapper.seek(nativeInstanceId, rowNumber); } /** @@ -66,7 +66,7 @@ public boolean seek(int rowNumber) { * @return ArrowReader that iterate over current stripes */ public ArrowReader nextStripeReader(long batchSize) { - long stripeReaderId = jniWrapper.nextStripeReader(id, batchSize); + long stripeReaderId = jniWrapper.nextStripeReader(nativeInstanceId, batchSize); if (stripeReaderId < 0) { return null; } @@ -80,11 +80,11 @@ public ArrowReader nextStripeReader(long batchSize) { * @return number of stripes */ public int getNumberOfStripes() { - return jniWrapper.getNumberOfStripes(id); + return jniWrapper.getNumberOfStripes(nativeInstanceId); } @Override public void close() throws Exception { - jniWrapper.close(id); + jniWrapper.close(nativeInstanceId); } } diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java index 69dcfdc553d..457d25e4f74 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReferenceManager.java @@ -102,18 +102,7 @@ public ArrowBuf deriveBuffer(ArrowBuf sourceBuffer, int index, int length) { @Override public OwnershipTransferResult transferOwnership(ArrowBuf sourceBuffer, BufferAllocator targetAllocator) { - retain(); - return new OwnershipTransferResult() { - @Override - public boolean getAllocationFit() { - return false; - } - - @Override - public ArrowBuf getTransferredBuffer() { - return sourceBuffer; - } - }; + throw new UnsupportedOperationException(); } @Override diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java index 1f1288eff6f..c69e74ad352 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcStripeReader.java @@ -43,23 +43,22 @@ public class OrcStripeReader extends ArrowReader { /** * reference to native stripe reader instance. */ - private final long id; - private MessageChannelReader schemaReader; + private final long nativeInstanceId; /** * Construct a new instance. - * @param id id of the stripe reader instance, obtained by + * @param nativeInstanceId nativeInstanceId of the stripe reader instance, obtained by * calling nextStripeReader from OrcReaderJniWrapper * @param allocator memory allocator for accounting. */ - OrcStripeReader(long id, BufferAllocator allocator) { + OrcStripeReader(long nativeInstanceId, BufferAllocator allocator) { super(allocator); - this.id = id; + this.nativeInstanceId = nativeInstanceId; } @Override public boolean loadNextBatch() throws IOException { - OrcRecordBatch recordBatch = OrcStripeReaderJniWrapper.next(id); + OrcRecordBatch recordBatch = OrcStripeReaderJniWrapper.next(nativeInstanceId); if (recordBatch == null) { return false; } @@ -91,28 +90,29 @@ public long bytesRead() { @Override protected void closeReadSource() throws IOException { - OrcStripeReaderJniWrapper.close(id); + OrcStripeReaderJniWrapper.close(nativeInstanceId); } @Override protected Schema readSchema() throws IOException { - byte[] schemaBytes = OrcStripeReaderJniWrapper.getSchema(id); - schemaReader = new MessageChannelReader( - new ReadChannel( - new ByteArrayReadableSeekableByteChannel(schemaBytes)), allocator); + byte[] schemaBytes = OrcStripeReaderJniWrapper.getSchema(nativeInstanceId); - MessageResult result = schemaReader.readNext(); - schemaReader.close(); + try (MessageChannelReader schemaReader = + new MessageChannelReader( + new ReadChannel( + new ByteArrayReadableSeekableByteChannel(schemaBytes)), allocator)) { - if (result == null) { - throw new IOException("Unexpected end of input. Missing schema."); - } + MessageResult result = schemaReader.readNext(); + if (result == null) { + throw new IOException("Unexpected end of input. Missing schema."); + } - if (result.getMessage().headerType() != MessageHeader.Schema) { - throw new IOException("Expected schema but header was " + result.getMessage().headerType()); - } + if (result.getMessage().headerType() != MessageHeader.Schema) { + throw new IOException("Expected schema but header was " + result.getMessage().headerType()); + } - return MessageSerializer.deserializeSchema(result.getMessage()); + return MessageSerializer.deserializeSchema(result.getMessage()); + } } @Override From 706c8dcd5a3114dcc729e822dc5d87cf613ce7fa Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Wed, 5 Jun 2019 17:23:49 +0800 Subject: [PATCH 19/21] resolve comments --- cpp/src/jni/orc/jni_wrapper.cpp | 56 ++++++++++++------- .../apache/arrow/adapter/orc/OrcJniUtils.java | 2 +- .../apache/arrow/adapter/orc/OrcReader.java | 8 +-- .../adapter/orc/OrcReaderJniWrapper.java | 2 +- 4 files changed, 41 insertions(+), 27 deletions(-) diff --git a/cpp/src/jni/orc/jni_wrapper.cpp b/cpp/src/jni/orc/jni_wrapper.cpp index 56eff805b21..f18bacc864d 100644 --- a/cpp/src/jni/orc/jni_wrapper.cpp +++ b/cpp/src/jni/orc/jni_wrapper.cpp @@ -35,7 +35,8 @@ using ORCFileReader = arrow::adapters::orc::ORCFileReader; using RecordBatchReader = arrow::RecordBatchReader; static jclass io_exception_class; -static jclass exception_class; +static jclass illegal_access_exception_class; +static jclass illegal_argument_exception_class; static jclass orc_field_node_class; static jmethodID orc_field_node_constructor; @@ -66,7 +67,7 @@ jmethodID GetMethodID(JNIEnv* env, jclass this_class, const char* name, const ch if (ret == nullptr) { std::string error_message = "Unable to find method " + std::string(name) + " within signature" + std::string(sig); - env->ThrowNew(exception_class, error_message.c_str()); + env->ThrowNew(illegal_access_exception_class, error_message.c_str()); } return ret; @@ -81,6 +82,26 @@ std::string JStringToCString(JNIEnv* env, jstring string) { return std::string(buffer.data(), clen); } +std::shared_ptr GetFileReader(JNIEnv* env, jlong id) { + auto reader = orc_reader_holder_.Lookup(id); + if (!reader) { + std::string error_message = "invalid reader id " + std::to_string(id); + env->ThrowNew(illegal_argument_exception_class, error_message.c_str()); + } + + return reader; +} + +std::shared_ptr GetStripeReader(JNIEnv* env, jlong id) { + auto reader = orc_stripe_reader_holder_.Lookup(id); + if (!reader) { + std::string error_message = "invalid stripe reader id " + std::to_string(id); + env->ThrowNew(illegal_argument_exception_class, error_message.c_str()); + } + + return reader; +} + #ifdef __cplusplus extern "C" { #endif @@ -92,7 +113,10 @@ jint JNI_OnLoad(JavaVM* vm, void* reserved) { } io_exception_class = CreateGlobalClassReference(env, "Ljava/io/IOException;"); - exception_class = CreateGlobalClassReference(env, "Ljava/lang/Exception;"); + illegal_access_exception_class = + CreateGlobalClassReference(env, "Ljava/lang/IllegalAccessException;"); + illegal_argument_exception_class = + CreateGlobalClassReference(env, "Ljava/lang/IllegalArgumentException;"); orc_field_node_class = CreateGlobalClassReference(env, "Lorg/apache/arrow/adapter/orc/OrcFieldNode;"); @@ -118,7 +142,8 @@ void JNI_OnUnload(JavaVM* vm, void* reserved) { JNIEnv* env; vm->GetEnv(reinterpret_cast(&env), JNI_VERSION); env->DeleteGlobalRef(io_exception_class); - env->DeleteGlobalRef(exception_class); + env->DeleteGlobalRef(illegal_access_exception_class); + env->DeleteGlobalRef(illegal_argument_exception_class); env->DeleteGlobalRef(orc_field_node_class); env->DeleteGlobalRef(orc_memory_class); env->DeleteGlobalRef(record_batch_class); @@ -165,7 +190,7 @@ JNIEXPORT void JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_clo JNIEXPORT jboolean JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_seek( JNIEnv* env, jobject this_obj, jlong id, jint row_number) { - auto reader = orc_reader_holder_.Lookup(id); + auto reader = GetFileReader(env, id); return reader->Seek(row_number).ok(); } @@ -173,7 +198,7 @@ JNIEXPORT jint JNICALL Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_getNumberOfStripes(JNIEnv* env, jobject this_obj, jlong id) { - auto reader = orc_reader_holder_.Lookup(id); + auto reader = GetFileReader(env, id); return reader->NumberOfStripes(); } @@ -182,11 +207,7 @@ Java_org_apache_arrow_adapter_orc_OrcReaderJniWrapper_nextStripeReader(JNIEnv* e jobject this_obj, jlong id, jlong batch_size) { - auto reader = orc_reader_holder_.Lookup(id); - if (!reader) { - std::string error_message = "invalid reader id " + std::to_string(id); - env->ThrowNew(exception_class, error_message.c_str()); - } + auto reader = GetFileReader(env, id); std::shared_ptr stripe_reader; auto status = reader->NextStripeReader(batch_size, &stripe_reader); @@ -206,11 +227,7 @@ JNIEXPORT jbyteArray JNICALL Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_getSchema(JNIEnv* env, jclass this_cls, jlong id) { - auto stripe_reader = orc_stripe_reader_holder_.Lookup(id); - if (!stripe_reader) { - std::string error_message = "invalid stripe reader id " + std::to_string(id); - env->ThrowNew(exception_class, error_message.c_str()); - } + auto stripe_reader = GetStripeReader(env, id); auto schema = stripe_reader->schema(); @@ -231,11 +248,7 @@ JNIEXPORT jobject JNICALL Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_next(JNIEnv* env, jclass this_cls, jlong id) { - auto stripe_reader = orc_stripe_reader_holder_.Lookup(id); - if (!stripe_reader) { - std::string error_message = "invalid stripe reader id " + std::to_string(id); - env->ThrowNew(exception_class, error_message.c_str()); - } + auto stripe_reader = GetStripeReader(env, id); std::shared_ptr record_batch; auto status = stripe_reader->ReadNext(&record_batch); @@ -245,6 +258,7 @@ Java_org_apache_arrow_adapter_orc_OrcStripeReaderJniWrapper_next(JNIEnv* env, auto schema = stripe_reader->schema(); + // TODO: ARROW-4714 Ensure JVM has sufficient capacity to create local references // create OrcFieldNode[] jobjectArray field_array = env->NewObjectArray(schema->num_fields(), orc_field_node_class, nullptr); diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java index 721827bc91a..600569be7c7 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcJniUtils.java @@ -34,7 +34,7 @@ class OrcJniUtils { private OrcJniUtils() {} static void loadOrcAdapterLibraryFromJar() - throws IOException { + throws IOException, IllegalAccessException { synchronized (OrcJniUtils.class) { if (!isLoaded) { final String libraryToLoad = System.mapLibraryName(LIBRARY_NAME); diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java index 7f95f65f775..366489fe562 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReader.java @@ -43,7 +43,7 @@ public class OrcReader implements AutoCloseable { * @param allocator allocator provided to ArrowReader. * @throws IOException throws exception in case of file not found */ - public OrcReader(String filePath, BufferAllocator allocator) throws IOException { + public OrcReader(String filePath, BufferAllocator allocator) throws IOException, IllegalAccessException { this.allocator = allocator; this.jniWrapper = OrcReaderJniWrapper.getInstance(); this.nativeInstanceId = jniWrapper.open(filePath); @@ -55,7 +55,7 @@ public OrcReader(String filePath, BufferAllocator allocator) throws IOException * @param rowNumber the rows number to seek * @return true if seek operation is succeeded */ - public boolean seek(int rowNumber) { + public boolean seek(int rowNumber) throws IllegalArgumentException { return jniWrapper.seek(nativeInstanceId, rowNumber); } @@ -65,7 +65,7 @@ public boolean seek(int rowNumber) { * @param batchSize the number of rows loaded on each iteration * @return ArrowReader that iterate over current stripes */ - public ArrowReader nextStripeReader(long batchSize) { + public ArrowReader nextStripeReader(long batchSize) throws IllegalArgumentException { long stripeReaderId = jniWrapper.nextStripeReader(nativeInstanceId, batchSize); if (stripeReaderId < 0) { return null; @@ -79,7 +79,7 @@ public ArrowReader nextStripeReader(long batchSize) { * * @return number of stripes */ - public int getNumberOfStripes() { + public int getNumberOfStripes() throws IllegalArgumentException { return jniWrapper.getNumberOfStripes(nativeInstanceId); } diff --git a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java index 6c5e30a17d0..ff449c343c4 100644 --- a/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java +++ b/java/adapter/orc/src/main/java/org/apache/arrow/adapter/orc/OrcReaderJniWrapper.java @@ -26,7 +26,7 @@ class OrcReaderJniWrapper { private static volatile OrcReaderJniWrapper INSTANCE; - static OrcReaderJniWrapper getInstance() throws IOException { + static OrcReaderJniWrapper getInstance() throws IOException, IllegalAccessException { if (INSTANCE == null) { synchronized (OrcReaderJniWrapper.class) { if (INSTANCE == null) { From 44b54203bf7615ac8ecf59765eb594ac6c6ca70f Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Thu, 6 Jun 2019 12:10:02 +0800 Subject: [PATCH 20/21] make sure lookup operation are performed under lock --- cpp/src/jni/orc/concurrent_map.h | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/cpp/src/jni/orc/concurrent_map.h b/cpp/src/jni/orc/concurrent_map.h index cc875030b7c..57fd9c4e76c 100644 --- a/cpp/src/jni/orc/concurrent_map.h +++ b/cpp/src/jni/orc/concurrent_map.h @@ -50,12 +50,8 @@ class ConcurrentMap { } Holder Lookup(jlong module_id) { - auto it = map_.find(module_id); - if (it != map_.end()) { - return it->second; - } std::lock_guard lock(mtx_); - it = map_.find(module_id); + auto it = map_.find(module_id); if (it != map_.end()) { return it->second; } @@ -68,7 +64,7 @@ class ConcurrentMap { } private: - // starting value of the module_id. + // initialize the module id starting value to a number greater than zero. static constexpr int init_module_id_ = 4; int64_t module_id_; From 41592bf049e38080b3fcdbce9792587207b37905 Mon Sep 17 00:00:00 2001 From: Yurui Zhou Date: Thu, 6 Jun 2019 14:22:51 +0800 Subject: [PATCH 21/21] minor doc fix --- cpp/src/jni/orc/concurrent_map.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cpp/src/jni/orc/concurrent_map.h b/cpp/src/jni/orc/concurrent_map.h index 57fd9c4e76c..9ca2fc825df 100644 --- a/cpp/src/jni/orc/concurrent_map.h +++ b/cpp/src/jni/orc/concurrent_map.h @@ -64,7 +64,8 @@ class ConcurrentMap { } private: - // initialize the module id starting value to a number greater than zero. + // Initialize the module id starting value to a number greater than zero + // to allow for easier debugging of uninitialized java variables. static constexpr int init_module_id_ = 4; int64_t module_id_;