You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/06/29 14:34:39 UTC

[GitHub] [arrow] lidavidm opened a new pull request, #13465: ARROW-16913: [Java] Implement ArrowArrayStream

lidavidm opened a new pull request, #13465:
URL: https://github.com/apache/arrow/pull/13465

   Implements ArrowArrayStream for Java. The equivalent Java-side interface chosen is ArrowReader.
   
   Also:
   - Fixes a couple of JDK9 compatibility issues I ran into. I _think_ these will not normally affect people except during development (I think because I was mixing IntelliJ and Maven).
   - Manually clang-format the C++ code. Clean up some things to match Arrow convention and remove some unused declarations.
   - Extends the DictionaryProvider interface. This is a potentially breaking change; we could make the method default (and raise an exception) instead.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
pitrou commented on code in PR #13465:
URL: https://github.com/apache/arrow/pull/13465#discussion_r910102218


##########
java/c/src/main/cpp/jni_wrapper.cc:
##########
@@ -148,16 +176,99 @@ void release_exported(T* base) {
   // Mark released
   base->release = nullptr;
 }
+
+int ArrowArrayStreamGetSchema(ArrowArrayStream* stream, ArrowSchema* out) {
+  assert(stream->private_data != nullptr);
+  InnerPrivateData* private_data =
+      reinterpret_cast<InnerPrivateData*>(stream->private_data);
+  JNIEnvGuard guard(private_data->vm_);
+  JNIEnv* env = guard.env();
+
+  const long out_addr = static_cast<long>(reinterpret_cast<uintptr_t>(out));
+  const int err_code = env->CallIntMethod(private_data->j_private_data_,
+                                          kPrivateDataGetSchemaMethod, out_addr);
+  if (env->ExceptionCheck()) {
+    env->ExceptionDescribe();

Review Comment:
   If there's an exception, should it perhaps participate in `last_error_`?



##########
java/c/src/main/cpp/jni_wrapper.cc:
##########
@@ -148,16 +176,99 @@ void release_exported(T* base) {
   // Mark released
   base->release = nullptr;
 }
+
+int ArrowArrayStreamGetSchema(ArrowArrayStream* stream, ArrowSchema* out) {
+  assert(stream->private_data != nullptr);
+  InnerPrivateData* private_data =
+      reinterpret_cast<InnerPrivateData*>(stream->private_data);
+  JNIEnvGuard guard(private_data->vm_);
+  JNIEnv* env = guard.env();
+
+  const long out_addr = static_cast<long>(reinterpret_cast<uintptr_t>(out));
+  const int err_code = env->CallIntMethod(private_data->j_private_data_,
+                                          kPrivateDataGetSchemaMethod, out_addr);
+  if (env->ExceptionCheck()) {
+    env->ExceptionDescribe();
+    env->ExceptionClear();
+    return EIO;
+  }
+  return err_code;
+}
+
+int ArrowArrayStreamGetNext(ArrowArrayStream* stream, ArrowArray* out) {
+  assert(stream->private_data != nullptr);
+  InnerPrivateData* private_data =
+      reinterpret_cast<InnerPrivateData*>(stream->private_data);
+  JNIEnvGuard guard(private_data->vm_);
+  JNIEnv* env = guard.env();
+
+  const long out_addr = static_cast<long>(reinterpret_cast<uintptr_t>(out));
+  const int err_code = env->CallIntMethod(private_data->j_private_data_,
+                                          kPrivateDataGetNextMethod, out_addr);
+  if (env->ExceptionCheck()) {
+    env->ExceptionDescribe();
+    env->ExceptionClear();
+    return EIO;
+  }
+  return err_code;
+}
+
+const char* ArrowArrayStreamGetLastError(ArrowArrayStream* stream) {
+  assert(stream->private_data != nullptr);
+  InnerPrivateData* private_data =
+      reinterpret_cast<InnerPrivateData*>(stream->private_data);
+  JNIEnvGuard guard(private_data->vm_);
+  JNIEnv* env = guard.env();
+
+  jobject error_data =
+      env->GetObjectField(private_data->j_private_data_, kPrivateDataLastErrorField);
+  if (!error_data) return nullptr;
+
+  auto arr = reinterpret_cast<jbyteArray>(error_data);
+  jbyte* error_bytes = env->GetByteArrayElements(arr, nullptr);
+  if (!error_bytes) return nullptr;
+
+  char* error_str = reinterpret_cast<char*>(error_bytes);
+  private_data->last_error_ = std::string(error_str, std::strlen(error_str));
+
+  env->ReleaseByteArrayElements(arr, error_bytes, JNI_ABORT);
+  return private_data->last_error_.c_str();
+}
+
+void ArrowArrayStreamRelease(ArrowArrayStream* stream) {
+  // This should not be called on already released structure
+  assert(stream->release != nullptr);
+  // Release all data directly owned by the struct
+  InnerPrivateData* private_data =
+      reinterpret_cast<InnerPrivateData*>(stream->private_data);
+
+  JNIEnvGuard guard(private_data->vm_);
+  JNIEnv* env = guard.env();
+
+  env->CallObjectMethod(private_data->j_private_data_, kPrivateDataCloseMethod);
+  if (env->ExceptionCheck()) {
+    env->ExceptionDescribe();
+    env->ExceptionClear();
+    ThrowPendingException("Error calling close of private data");

Review Comment:
   Is this right? The release callback could be called from any context, such as a Python thread or R interpreter. In those contexts, a C++ exception would probably crash the process (or silently exit the thread)?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
pitrou commented on code in PR #13465:
URL: https://github.com/apache/arrow/pull/13465#discussion_r910142254


##########
java/c/src/main/java/org/apache/arrow/c/ArrayStreamExporter.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.c;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.arrow.c.jni.JniWrapper;
+import org.apache.arrow.c.jni.PrivateData;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.ipc.ArrowReader;
+import org.apache.arrow.vector.types.pojo.Schema;
+
+/**
+ * Utility to export an {@link ArrowReader} as an ArrowArrayStream.
+ */
+final class ArrayStreamExporter {
+  private final BufferAllocator allocator;
+
+  ArrayStreamExporter(BufferAllocator allocator) {
+    this.allocator = allocator;
+  }
+
+  /**
+   * Java-side state for the exported stream.
+   */
+  static class ExportedArrayStreamPrivateData implements PrivateData {
+    final BufferAllocator allocator;
+    final ArrowReader reader;
+    int nextDictionary;

Review Comment:
   This member doesn't seem used anymore, or am I missing something?



##########
java/c/src/main/cpp/jni_wrapper.cc:
##########
@@ -148,16 +176,99 @@ void release_exported(T* base) {
   // Mark released
   base->release = nullptr;
 }
+
+int ArrowArrayStreamGetSchema(ArrowArrayStream* stream, ArrowSchema* out) {
+  assert(stream->private_data != nullptr);
+  InnerPrivateData* private_data =
+      reinterpret_cast<InnerPrivateData*>(stream->private_data);
+  JNIEnvGuard guard(private_data->vm_);
+  JNIEnv* env = guard.env();
+
+  const long out_addr = static_cast<long>(reinterpret_cast<uintptr_t>(out));

Review Comment:
   Also, according to the [JNI spec](https://docs.oracle.com/javase/7/docs/technotes/guides/jni/spec/types.html), a `jlong` is always 64 bits, so perhaps we should use `jlong` or simply `int64_t` here?



##########
java/c/src/main/cpp/jni_wrapper.cc:
##########
@@ -148,16 +176,99 @@ void release_exported(T* base) {
   // Mark released
   base->release = nullptr;
 }
+
+int ArrowArrayStreamGetSchema(ArrowArrayStream* stream, ArrowSchema* out) {
+  assert(stream->private_data != nullptr);
+  InnerPrivateData* private_data =
+      reinterpret_cast<InnerPrivateData*>(stream->private_data);
+  JNIEnvGuard guard(private_data->vm_);
+  JNIEnv* env = guard.env();
+
+  const long out_addr = static_cast<long>(reinterpret_cast<uintptr_t>(out));

Review Comment:
   I suppose this doesn't work on 64-bit Windows? `long` is 32 bits there...



##########
java/c/src/main/java/org/apache/arrow/c/Data.java:
##########
@@ -314,4 +325,14 @@ public static VectorSchemaRoot importVectorSchemaRoot(BufferAllocator allocator,
     }
     return vsr;
   }
+
+  /**
+   * Import an ArrowArrayStream as an {@link ArrowReader}.
+   * @param allocator Buffer allocator for allocating the output data.
+   * @param stream C stream interface struct to import.
+   * @return Imported reader
+   */
+  public static ArrowReader importStream(BufferAllocator allocator, ArrowArrayStream stream) {

Review Comment:
   Is there a reason for the naming discrepancy (`importStream` vs. `exportArrayStream`)?



##########
java/c/src/main/cpp/jni_wrapper.cc:
##########
@@ -54,23 +53,52 @@ void ThrowPendingException(const std::string& message) {
 
 void JniThrow(std::string message) { ThrowPendingException(message); }
 
+jclass CreateGlobalClassReference(JNIEnv* env, const char* class_name) {
+  jclass local_class = env->FindClass(class_name);
+  if (!local_class) {
+    std::string message = "Could not find class ";
+    message += class_name;
+    ThrowPendingException(message);
+  }
+  jclass global_class = (jclass)env->NewGlobalRef(local_class);
+  if (!local_class) {

Review Comment:
   Is this a mistake?
   ```suggestion
     if (!global_class) {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm commented on pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
lidavidm commented on PR #13465:
URL: https://github.com/apache/arrow/pull/13465#issuecomment-1171342321

   Well, the Python-side reference is the Python reader object itself. But close() should be wired up to call the new RecordBatchReader::Close() so we can at least explicitly call the release callback at a suitable time.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] amol- merged pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
amol- merged PR #13465:
URL: https://github.com/apache/arrow/pull/13465


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
pitrou commented on PR #13465:
URL: https://github.com/apache/arrow/pull/13465#issuecomment-1171147151

   @lwhite1 Would you like to take a look?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm closed pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
lidavidm closed pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream
URL: https://github.com/apache/arrow/pull/13465


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm commented on a diff in pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13465:
URL: https://github.com/apache/arrow/pull/13465#discussion_r910225998


##########
java/c/src/main/java/org/apache/arrow/c/ArrayStreamExporter.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.c;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.arrow.c.jni.JniWrapper;
+import org.apache.arrow.c.jni.PrivateData;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.ipc.ArrowReader;
+import org.apache.arrow.vector.types.pojo.Schema;
+
+/**
+ * Utility to export an {@link ArrowReader} as an ArrowArrayStream.
+ */
+final class ArrayStreamExporter {
+  private final BufferAllocator allocator;
+
+  ArrayStreamExporter(BufferAllocator allocator) {
+    this.allocator = allocator;
+  }
+
+  /**
+   * Java-side state for the exported stream.
+   */
+  static class ExportedArrayStreamPrivateData implements PrivateData {
+    final BufferAllocator allocator;
+    final ArrowReader reader;
+    int nextDictionary;

Review Comment:
   Ah it's not used. I missed this when backing out a change.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm commented on pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
lidavidm commented on PR #13465:
URL: https://github.com/apache/arrow/pull/13465#issuecomment-1171338313

   Hmm, there's a possible minor bug between PyArrow/C++/Java: Python can keep a reference to the reader until interpreter shutdown (at which point the JVM has been shut down), and then collects the reader. This frees the `shared_ptr<RecordBatchReader>`; the implementation is `ArrayStreamBatchReader` whose destructor unconditionally calls ArrowArrayStreamRelease. (It doesn't implement Close() since this was just added). This calls into the JNI code, which tries to attach the JVM, fails, throws an uncaught exception, and dies.
   
   Changes needed:
   - Implement Close() properly so we can explicitly free
   - Don't unconditionally call ArrowArrayStreamRelease
   - In the JNI side, fail gracefully if we can't attach the JVM (warn instead of abort?)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
pitrou commented on PR #13465:
URL: https://github.com/apache/arrow/pull/13465#issuecomment-1171340398

   > Python can keep a reference to the reader until interpreter shutdown (at which point the JVM has been shut down), and then collects the reader
   
   Can Python perhaps release that reference once close() is called?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
pitrou commented on PR #13465:
URL: https://github.com/apache/arrow/pull/13465#issuecomment-1171342389

   Though the Java improvements are welcome as well. We should probably try to do both.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm commented on pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
lidavidm commented on PR #13465:
URL: https://github.com/apache/arrow/pull/13465#issuecomment-1171154089

   > For the record, did you try to use this to communicate with e.g. PyArrow?
   
   I have not yet, I need to give this a try: https://arrow.apache.org/docs/dev/python/integration/python_java.html#java-to-python-communication-using-the-c-data-interface
   
   and actually, I'll extend the doc page there as well.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm commented on pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
lidavidm commented on PR #13465:
URL: https://github.com/apache/arrow/pull/13465#issuecomment-1171154656

   …wow, whatever GitHub did to their UI is rather frustrating.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm closed pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
lidavidm closed pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream
URL: https://github.com/apache/arrow/pull/13465


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm commented on pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
lidavidm commented on PR #13465:
URL: https://github.com/apache/arrow/pull/13465#issuecomment-1171342464

   …


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
pitrou commented on code in PR #13465:
URL: https://github.com/apache/arrow/pull/13465#discussion_r911183043


##########
java/c/src/main/cpp/jni_wrapper.cc:
##########
@@ -162,17 +163,25 @@ void release_exported(T* base) {
   InnerPrivateData* private_data =
       reinterpret_cast<InnerPrivateData*>(base->private_data);
 
-  JNIEnvGuard guard(private_data->vm_);
-  JNIEnv* env = guard.env();
-
-  env->CallObjectMethod(private_data->j_private_data_, kPrivateDataCloseMethod);
-  if (env->ExceptionCheck()) {
-    // Can't signal this to caller, so log and then try to free things
-    // as best we can
-    env->ExceptionDescribe();
-    env->ExceptionClear();
+  // It is possible for the JVM to be shut down when this is called;
+  // guard against that.  Example: Python code using JPype may shut
+  // down the JVM before releasing the stream.
+  try {
+    JNIEnvGuard guard(private_data->vm_);
+    JNIEnv* env = guard.env();
+
+    env->CallObjectMethod(private_data->j_private_data_, kPrivateDataCloseMethod);
+    if (env->ExceptionCheck()) {
+      // Can't signal this to caller, so log and then try to free things
+      // as best we can
+      env->ExceptionDescribe();
+      env->ExceptionClear();
+    }
+    env->DeleteGlobalRef(private_data->j_private_data_);
+  } catch (const JniPendingException& e) {
+    std::cerr << "WARNING: Failed to release Java C Data resource: " << e.what()

Review Comment:
   Do we have access to `ARROW_LOG(WARNING)` here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
pitrou commented on PR #13465:
URL: https://github.com/apache/arrow/pull/13465#issuecomment-1171384411

   @amol- Do you want to take a look at the doc additions?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm commented on a diff in pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13465:
URL: https://github.com/apache/arrow/pull/13465#discussion_r911183701


##########
java/c/src/main/cpp/jni_wrapper.cc:
##########
@@ -162,17 +163,25 @@ void release_exported(T* base) {
   InnerPrivateData* private_data =
       reinterpret_cast<InnerPrivateData*>(base->private_data);
 
-  JNIEnvGuard guard(private_data->vm_);
-  JNIEnv* env = guard.env();
-
-  env->CallObjectMethod(private_data->j_private_data_, kPrivateDataCloseMethod);
-  if (env->ExceptionCheck()) {
-    // Can't signal this to caller, so log and then try to free things
-    // as best we can
-    env->ExceptionDescribe();
-    env->ExceptionClear();
+  // It is possible for the JVM to be shut down when this is called;
+  // guard against that.  Example: Python code using JPype may shut
+  // down the JVM before releasing the stream.
+  try {
+    JNIEnvGuard guard(private_data->vm_);
+    JNIEnv* env = guard.env();
+
+    env->CallObjectMethod(private_data->j_private_data_, kPrivateDataCloseMethod);
+    if (env->ExceptionCheck()) {
+      // Can't signal this to caller, so log and then try to free things
+      // as best we can
+      env->ExceptionDescribe();
+      env->ExceptionClear();
+    }
+    env->DeleteGlobalRef(private_data->j_private_data_);
+  } catch (const JniPendingException& e) {
+    std::cerr << "WARNING: Failed to release Java C Data resource: " << e.what()

Review Comment:
   No, since this isn't linked against libarrow/depends only on the ABI header and JNI.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13465:
URL: https://github.com/apache/arrow/pull/13465#issuecomment-1170082256

   https://issues.apache.org/jira/browse/ARROW-16913


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm commented on a diff in pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13465:
URL: https://github.com/apache/arrow/pull/13465#discussion_r910225535


##########
java/c/src/main/cpp/jni_wrapper.cc:
##########
@@ -148,16 +176,99 @@ void release_exported(T* base) {
   // Mark released
   base->release = nullptr;
 }
+
+int ArrowArrayStreamGetSchema(ArrowArrayStream* stream, ArrowSchema* out) {
+  assert(stream->private_data != nullptr);
+  InnerPrivateData* private_data =
+      reinterpret_cast<InnerPrivateData*>(stream->private_data);
+  JNIEnvGuard guard(private_data->vm_);
+  JNIEnv* env = guard.env();
+
+  const long out_addr = static_cast<long>(reinterpret_cast<uintptr_t>(out));
+  const int err_code = env->CallIntMethod(private_data->j_private_data_,
+                                          kPrivateDataGetSchemaMethod, out_addr);
+  if (env->ExceptionCheck()) {
+    env->ExceptionDescribe();
+    env->ExceptionClear();
+    return EIO;
+  }
+  return err_code;
+}
+
+int ArrowArrayStreamGetNext(ArrowArrayStream* stream, ArrowArray* out) {
+  assert(stream->private_data != nullptr);
+  InnerPrivateData* private_data =
+      reinterpret_cast<InnerPrivateData*>(stream->private_data);
+  JNIEnvGuard guard(private_data->vm_);
+  JNIEnv* env = guard.env();
+
+  const long out_addr = static_cast<long>(reinterpret_cast<uintptr_t>(out));
+  const int err_code = env->CallIntMethod(private_data->j_private_data_,
+                                          kPrivateDataGetNextMethod, out_addr);
+  if (env->ExceptionCheck()) {
+    env->ExceptionDescribe();
+    env->ExceptionClear();
+    return EIO;
+  }
+  return err_code;
+}
+
+const char* ArrowArrayStreamGetLastError(ArrowArrayStream* stream) {
+  assert(stream->private_data != nullptr);
+  InnerPrivateData* private_data =
+      reinterpret_cast<InnerPrivateData*>(stream->private_data);
+  JNIEnvGuard guard(private_data->vm_);
+  JNIEnv* env = guard.env();
+
+  jobject error_data =
+      env->GetObjectField(private_data->j_private_data_, kPrivateDataLastErrorField);
+  if (!error_data) return nullptr;
+
+  auto arr = reinterpret_cast<jbyteArray>(error_data);
+  jbyte* error_bytes = env->GetByteArrayElements(arr, nullptr);
+  if (!error_bytes) return nullptr;
+
+  char* error_str = reinterpret_cast<char*>(error_bytes);
+  private_data->last_error_ = std::string(error_str, std::strlen(error_str));
+
+  env->ReleaseByteArrayElements(arr, error_bytes, JNI_ABORT);
+  return private_data->last_error_.c_str();
+}
+
+void ArrowArrayStreamRelease(ArrowArrayStream* stream) {
+  // This should not be called on already released structure
+  assert(stream->release != nullptr);
+  // Release all data directly owned by the struct
+  InnerPrivateData* private_data =
+      reinterpret_cast<InnerPrivateData*>(stream->private_data);
+
+  JNIEnvGuard guard(private_data->vm_);
+  JNIEnv* env = guard.env();
+
+  env->CallObjectMethod(private_data->j_private_data_, kPrivateDataCloseMethod);
+  if (env->ExceptionCheck()) {
+    env->ExceptionDescribe();
+    env->ExceptionClear();
+    ThrowPendingException("Error calling close of private data");

Review Comment:
   ah, you're right. The existing handler has this issue too. I'll remove the throw. (Actually here I suppose we should do our best to free resources in C++/Java regardless.)



##########
java/c/src/main/cpp/jni_wrapper.cc:
##########
@@ -148,16 +176,99 @@ void release_exported(T* base) {
   // Mark released
   base->release = nullptr;
 }
+
+int ArrowArrayStreamGetSchema(ArrowArrayStream* stream, ArrowSchema* out) {
+  assert(stream->private_data != nullptr);
+  InnerPrivateData* private_data =
+      reinterpret_cast<InnerPrivateData*>(stream->private_data);
+  JNIEnvGuard guard(private_data->vm_);
+  JNIEnv* env = guard.env();
+
+  const long out_addr = static_cast<long>(reinterpret_cast<uintptr_t>(out));

Review Comment:
   Good catch, thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm commented on a diff in pull request #13465: ARROW-16913: [Java] Implement ArrowArrayStream

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13465:
URL: https://github.com/apache/arrow/pull/13465#discussion_r910224633


##########
java/c/src/main/cpp/jni_wrapper.cc:
##########
@@ -148,16 +176,99 @@ void release_exported(T* base) {
   // Mark released
   base->release = nullptr;
 }
+
+int ArrowArrayStreamGetSchema(ArrowArrayStream* stream, ArrowSchema* out) {
+  assert(stream->private_data != nullptr);
+  InnerPrivateData* private_data =
+      reinterpret_cast<InnerPrivateData*>(stream->private_data);
+  JNIEnvGuard guard(private_data->vm_);
+  JNIEnv* env = guard.env();
+
+  const long out_addr = static_cast<long>(reinterpret_cast<uintptr_t>(out));
+  const int err_code = env->CallIntMethod(private_data->j_private_data_,
+                                          kPrivateDataGetSchemaMethod, out_addr);
+  if (env->ExceptionCheck()) {
+    env->ExceptionDescribe();

Review Comment:
   Normally the JNI side sets the last error, the check here is just a last-resort safeguard. I suppose this can be refactored though: copy the Java-side error to the C++ side after get_next/get_stream, and get_last_error only has to return the C++-side error; then get_next/get_stream can also update last_error_ if it ends up catching a stray error.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org