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/03/17 20:48:30 UTC

[GitHub] [arrow] lidavidm opened a new pull request #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   This isn't perfect because some exceptions are meant to propagate to the client, some are unintended, and it's not always clear which is which, but this should at least make development easier.


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   Ah, interesting. That should work.


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   Note that CPython has a dedicated API to print unraisable errors, you may not need to use the logging module: https://docs.python.org/3/c-api/exceptions.html#c.PyErr_WriteUnraisable


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   I think most everything will stay the same, except we wouldn't filter the exceptions we log. (We could probably simplify things, though, if we used `self.__log`; I'm not a fan of exposing that to user code, even as a name-mangled property, but I don't feel too strongly about it either.)


-- 
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 change in pull request #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #12659:
URL: https://github.com/apache/arrow/pull/12659#discussion_r830267707



##########
File path: python/pyarrow/_flight.pyx
##########
@@ -1707,16 +1735,26 @@ cdef class ClientAuthSender(_Weakrefable):
         return result
 
 
-cdef CStatus _data_stream_next(void* self, CFlightPayload* payload) except *:
+cdef CStatus _data_stream_next(void* c_state,
+                               CFlightPayload* payload) except *:
     """Callback for implementing FlightDataStream in Python."""
+    server_state = <object> c_state
+    try:
+        if not isinstance(server_state.this, GeneratorStream):
+            raise RuntimeError(
+                "self object in callback is not GeneratorStream")
+        stream = <GeneratorStream> server_state.this
+        return _data_stream_do_next(stream, server_state, payload)
+    except Exception as e:
+        server_state.log_exception("do_get", e)

Review comment:
       Yes, basically DoGet returns a Stream and the Stream is consumed by the C++ side to finish off the call. So this is still within the DoGet RPC even if control flow is outside the Cython do_get code.




-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   Logging would be preferable to integrate with logging frameworks, though (e.g. if a service logs things to Datadog or some other vendor tool - that way uncaught exceptions could also be propagated)


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   Well, the changes in this PR can probably be vastly simplified if we add an error callback.


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   We certainly could; that would be a breaking change. (Well, we could place it in the `context`, actually - that should be OK.) I'd prefer to tackle that separately, though


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


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


-- 
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] edponce commented on a change in pull request #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #12659:
URL: https://github.com/apache/arrow/pull/12659#discussion_r829694665



##########
File path: python/pyarrow/_flight.pyx
##########
@@ -1490,20 +1491,47 @@ cdef class RecordBatchStream(FlightDataStream):
         self.data_source = data_source
         self.write_options = _get_options(options).c_options
 
-    cdef CFlightDataStream* to_stream(self) except *:
+    cdef CFlightDataStream* to_stream(self, logger) except *:
         cdef:
             shared_ptr[CRecordBatchReader] reader
         if isinstance(self.data_source, RecordBatchReader):
             reader = (<RecordBatchReader> self.data_source).reader
         elif isinstance(self.data_source, lib.Table):
-            table = (<Table> self.data_source).table
-            reader.reset(new TableBatchReader(deref(table)))
+            reader.reset(new TableBatchReader(
+                (<Table> self.data_source).sp_table))
         else:
             raise RuntimeError("Can't construct RecordBatchStream "
                                "from type {}".format(type(self.data_source)))
         return new CRecordBatchStream(reader, self.write_options)
 
 
+class _WithLogger(collections.namedtuple(
+        '_WithLogger', ['this', 'logger'])):
+    """Combine a server object with additional logging state.
+
+    This avoids exposing the logger in the public API (by storing it
+    on self, which even a name-mangled method can't avoid) and
+    potentially clashing with application code.
+    """
+
+    def log_exception(self, method, exc):
+        """Log an uncaught exception in the handler for an RPC method."""
+        if not self.logger:
+            return
+        # Don't log all kinds of errors. FlightError is explicitly for
+        # sending RPC errors; NotImplementedError is likely noise.
+        # Other errors are unclear, so log to be safe. For instance:
+        # both ValueError and ArrowInvalid map to INVALID_ARGUMENT,
+        # but ValueError is likely a stray exception (so log it) and
+        # ArrowInvalid is likely intentional (so don't)

Review comment:
       Extreme nit: `... (so don't).`  <-- period?

##########
File path: python/pyarrow/_flight.pyx
##########
@@ -1707,16 +1735,26 @@ cdef class ClientAuthSender(_Weakrefable):
         return result
 
 
-cdef CStatus _data_stream_next(void* self, CFlightPayload* payload) except *:
+cdef CStatus _data_stream_next(void* c_state,
+                               CFlightPayload* payload) except *:
     """Callback for implementing FlightDataStream in Python."""
+    server_state = <object> c_state
+    try:
+        if not isinstance(server_state.this, GeneratorStream):
+            raise RuntimeError(
+                "self object in callback is not GeneratorStream")
+        stream = <GeneratorStream> server_state.this
+        return _data_stream_do_next(stream, server_state, payload)
+    except Exception as e:
+        server_state.log_exception("do_get", e)

Review comment:
       Is this a "do_get" operation?
   [In `_do_get`](https://github.com/apache/arrow/pull/12659/files#diff-0ed358f5d42920d7f94cc500791976a2c158c4d72f4a6b231393534b2d13683bR1957), name is also "do_get".

##########
File path: python/pyarrow/_flight.pyx
##########
@@ -1866,41 +1919,49 @@ cdef CStatus _do_put(void* self, const CServerCallContext& context,
         FlightDescriptor descriptor = \
             FlightDescriptor.__new__(FlightDescriptor)
 
+    server_state = <object> c_state
     descriptor.descriptor = reader.get().descriptor()
     py_reader.reader.reset(reader.release())
     py_writer.writer.reset(writer.release())
     try:
-        (<object> self).do_put(ServerCallContext.wrap(context), descriptor,
-                               py_reader, py_writer)
+        server_state.this.do_put(ServerCallContext.wrap(context), descriptor,
+                                 py_reader, py_writer)
         return CStatus_OK()

Review comment:
       Nit: The general convention seems to be to place `return CStatus_OK()` at end of function.

##########
File path: python/pyarrow/_flight.pyx
##########
@@ -1910,27 +1971,32 @@ cdef CStatus _do_exchange(void* self, const CServerCallContext& context,
         FlightDescriptor descriptor = \
             FlightDescriptor.__new__(FlightDescriptor)
 
+    server_state = <object> c_state
     descriptor.descriptor = reader.get().descriptor()
     py_reader.reader.reset(reader.release())
     py_writer.writer.reset(writer.release())
     try:
-        (<object> self).do_exchange(ServerCallContext.wrap(context),
-                                    descriptor, py_reader, py_writer)
+        server_state.this.do_exchange(ServerCallContext.wrap(context),
+                                      descriptor, py_reader, py_writer)
         return CStatus_OK()

Review comment:
       Another inlined return statement.




-- 
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] edponce commented on pull request #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

Posted by GitBox <gi...@apache.org>.
edponce commented on pull request #12659:
URL: https://github.com/apache/arrow/pull/12659#issuecomment-1072038608


   This looks good. Simply left minor comments.


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   Hmm, it looks like you should skip the unraisable hook-based test on Python < 3.8 where the hook doesn't exist.
   
   Also, can you rebase to get the AppVeyor Flight crash fixed?


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   Thanks for the review!


-- 
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] edponce commented on pull request #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

Posted by GitBox <gi...@apache.org>.
edponce commented on pull request #12659:
URL: https://github.com/apache/arrow/pull/12659#issuecomment-1072038608


   This looks good. Simply left minor comments.


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   For that [`sys.unraisablehook()`](https://docs.python.org/3/library/sys.html#sys.unraisablehook) can be overriden. I have no preference personally, but not having to rely on a logger may make things easier compared to your current approach.


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   Perhaps we can provide such a callback too?


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   It's not clear to me what the intended purpose is. Why not log all exceptions?


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   Changed to use `PyErr_WriteUnraisable`. I still think this is orthogonal to adding an error callback, since either way we need/want to catch uncaught exceptions.
   
   Fun fact, the Cython generated code clears the current exception inside the `except` block, so it was failing to log the exception…that took way too long to figure out. 


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


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


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   Filed ARROW-16023 to migrate the server RPC handler API.


-- 
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 #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

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


   Certain exceptions are made to be raised to indicate errors to the client, and aren't meant to indicate an error condition on the server. This is rather confusing, unfortunately (hence why grpc-java differentiates the two and provides a callback to send errors to the client 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] edponce commented on a change in pull request #12659: ARROW-15909: [Python] Log uncaught exceptions in Flight RPC handlers

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #12659:
URL: https://github.com/apache/arrow/pull/12659#discussion_r829694665



##########
File path: python/pyarrow/_flight.pyx
##########
@@ -1490,20 +1491,47 @@ cdef class RecordBatchStream(FlightDataStream):
         self.data_source = data_source
         self.write_options = _get_options(options).c_options
 
-    cdef CFlightDataStream* to_stream(self) except *:
+    cdef CFlightDataStream* to_stream(self, logger) except *:
         cdef:
             shared_ptr[CRecordBatchReader] reader
         if isinstance(self.data_source, RecordBatchReader):
             reader = (<RecordBatchReader> self.data_source).reader
         elif isinstance(self.data_source, lib.Table):
-            table = (<Table> self.data_source).table
-            reader.reset(new TableBatchReader(deref(table)))
+            reader.reset(new TableBatchReader(
+                (<Table> self.data_source).sp_table))
         else:
             raise RuntimeError("Can't construct RecordBatchStream "
                                "from type {}".format(type(self.data_source)))
         return new CRecordBatchStream(reader, self.write_options)
 
 
+class _WithLogger(collections.namedtuple(
+        '_WithLogger', ['this', 'logger'])):
+    """Combine a server object with additional logging state.
+
+    This avoids exposing the logger in the public API (by storing it
+    on self, which even a name-mangled method can't avoid) and
+    potentially clashing with application code.
+    """
+
+    def log_exception(self, method, exc):
+        """Log an uncaught exception in the handler for an RPC method."""
+        if not self.logger:
+            return
+        # Don't log all kinds of errors. FlightError is explicitly for
+        # sending RPC errors; NotImplementedError is likely noise.
+        # Other errors are unclear, so log to be safe. For instance:
+        # both ValueError and ArrowInvalid map to INVALID_ARGUMENT,
+        # but ValueError is likely a stray exception (so log it) and
+        # ArrowInvalid is likely intentional (so don't)

Review comment:
       Extreme nit: `... (so don't).`  <-- period?

##########
File path: python/pyarrow/_flight.pyx
##########
@@ -1707,16 +1735,26 @@ cdef class ClientAuthSender(_Weakrefable):
         return result
 
 
-cdef CStatus _data_stream_next(void* self, CFlightPayload* payload) except *:
+cdef CStatus _data_stream_next(void* c_state,
+                               CFlightPayload* payload) except *:
     """Callback for implementing FlightDataStream in Python."""
+    server_state = <object> c_state
+    try:
+        if not isinstance(server_state.this, GeneratorStream):
+            raise RuntimeError(
+                "self object in callback is not GeneratorStream")
+        stream = <GeneratorStream> server_state.this
+        return _data_stream_do_next(stream, server_state, payload)
+    except Exception as e:
+        server_state.log_exception("do_get", e)

Review comment:
       Is this a "do_get" operation?
   [In `_do_get`](https://github.com/apache/arrow/pull/12659/files#diff-0ed358f5d42920d7f94cc500791976a2c158c4d72f4a6b231393534b2d13683bR1957), name is also "do_get".

##########
File path: python/pyarrow/_flight.pyx
##########
@@ -1866,41 +1919,49 @@ cdef CStatus _do_put(void* self, const CServerCallContext& context,
         FlightDescriptor descriptor = \
             FlightDescriptor.__new__(FlightDescriptor)
 
+    server_state = <object> c_state
     descriptor.descriptor = reader.get().descriptor()
     py_reader.reader.reset(reader.release())
     py_writer.writer.reset(writer.release())
     try:
-        (<object> self).do_put(ServerCallContext.wrap(context), descriptor,
-                               py_reader, py_writer)
+        server_state.this.do_put(ServerCallContext.wrap(context), descriptor,
+                                 py_reader, py_writer)
         return CStatus_OK()

Review comment:
       Nit: The general convention seems to be to place `return CStatus_OK()` at end of function.

##########
File path: python/pyarrow/_flight.pyx
##########
@@ -1910,27 +1971,32 @@ cdef CStatus _do_exchange(void* self, const CServerCallContext& context,
         FlightDescriptor descriptor = \
             FlightDescriptor.__new__(FlightDescriptor)
 
+    server_state = <object> c_state
     descriptor.descriptor = reader.get().descriptor()
     py_reader.reader.reset(reader.release())
     py_writer.writer.reset(writer.release())
     try:
-        (<object> self).do_exchange(ServerCallContext.wrap(context),
-                                    descriptor, py_reader, py_writer)
+        server_state.this.do_exchange(ServerCallContext.wrap(context),
+                                      descriptor, py_reader, py_writer)
         return CStatus_OK()

Review comment:
       Another inlined return statement.




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