You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ap...@apache.org on 2019/07/16 20:03:50 UTC

[arrow] branch master updated: ARROW-5930: [Python] Make Flight server init phase explicit

This is an automated email from the ASF dual-hosted git repository.

apitrou pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new ec78e13  ARROW-5930: [Python] Make Flight server init phase explicit
ec78e13 is described below

commit ec78e1306fb2c4573af2c6a8fc72c527501ebcfe
Author: Antoine Pitrou <an...@python.org>
AuthorDate: Tue Jul 16 22:03:33 2019 +0200

    ARROW-5930: [Python] Make Flight server init phase explicit
    
    Should fix the sporadic crashes in test_flight.py.
    
    Author: Antoine Pitrou <an...@python.org>
    
    Closes #4891 from pitrou/ARROW-5930-core-flight and squashes the following commits:
    
    842f1013c <Antoine Pitrou> ARROW-5930:  Make Flight server init phase explicit
---
 .travis.yml                                 |  2 +-
 ci/travis/after-failure.sh                  |  5 +++--
 ci/travis/before-install.sh                 |  5 ++++-
 ci/travis_script_python.sh                  |  5 ++++-
 cpp/src/arrow/flight/perf-server.cc         |  2 +-
 cpp/src/arrow/flight/server.cc              |  8 ++++++--
 cpp/src/arrow/flight/server.h               |  2 +-
 cpp/src/arrow/flight/test-util.cc           |  2 +-
 python/examples/flight/server.py            |  3 ++-
 python/pyarrow/_flight.pyx                  | 22 +++++++++++++++++-----
 python/pyarrow/includes/libarrow_flight.pxd |  2 +-
 python/pyarrow/tests/test_flight.py         | 19 +++++++------------
 12 files changed, 48 insertions(+), 29 deletions(-)

diff --git a/.travis.yml b/.travis.yml
index 49f9ea8..88ba1ef 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -120,7 +120,7 @@ matrix:
     # Only run Plasma tests with valgrind in one of the Python builds because
     # they are slow
     - export PLASMA_VALGRIND=1
-    - $TRAVIS_BUILD_DIR/ci/travis_script_python.sh 3.6 || travis_terminate 1
+    - $TRAVIS_BUILD_DIR/ci/travis_script_python.sh 3.6
     - $TRAVIS_BUILD_DIR/ci/travis_upload_cpp_coverage.sh
   - name: "[OS X] C++ w/ XCode 9.3"
     compiler: clang
diff --git a/ci/travis/after-failure.sh b/ci/travis/after-failure.sh
index 35b798f..9e77ec3 100755
--- a/ci/travis/after-failure.sh
+++ b/ci/travis/after-failure.sh
@@ -26,9 +26,10 @@ if [[ "${TRAVIS_OS_NAME}" = "osx" ]]; then
   ls -la ~/Library/Logs/DiagnosticReports/
   cat ~/Library/Logs/DiagnosticReports/*.crash
 else
-  COREFILE=$(find . -maxdepth 2 -name "core*" | head -n 1)
+  ls -fd1 /tmp/core.*
+  COREFILE=$(ls -fd1 /tmp/core.* | head -n 1)
   if [[ -f "$COREFILE" ]]; then
-    gdb -c "$COREFILE" example -ex "thread apply all bt" -ex "set pagination 0" -batch
+    gdb -c "$COREFILE" $TRAVIS_BUILD_DIR/current-exe -ex "thread apply all bt" -ex "set pagination 0" -batch
   fi
 fi
 
diff --git a/ci/travis/before-install.sh b/ci/travis/before-install.sh
index 02dc971..fd97b46 100755
--- a/ci/travis/before-install.sh
+++ b/ci/travis/before-install.sh
@@ -20,10 +20,13 @@ set -ex
 
 eval "${MATRIX_EVAL}"
 
-# Enables core files
+# Enable core files
 ulimit -c unlimited -S
 
 if [[ "${TRAVIS_OS_NAME}" == "linux" ]]; then
+  # Remove apport's core_pattern
+  sudo bash -c "echo '/tmp/core.%p.%E' > /proc/sys/kernel/core_pattern"
+
   echo -e 'Acquire::Retries 10; Acquire::http::Timeout \"20\";' | \
     sudo tee /etc/apt/apt.conf.d/99-travis-retry
   sudo apt-get update -qq
diff --git a/ci/travis_script_python.sh b/ci/travis_script_python.sh
index 8f50729..202c24f 100755
--- a/ci/travis_script_python.sh
+++ b/ci/travis_script_python.sh
@@ -192,11 +192,14 @@ if [ $TRAVIS_OS_NAME == "linux" ]; then
     sudo bash -c "echo 2048 > /proc/sys/vm/nr_hugepages"
 fi
 
+# For core dump analysis
+ln -sf `which python` $TRAVIS_BUILD_DIR/current-exe
+
 # Need to run tests from the source tree for Cython coverage and conftest.py
 if [ "$ARROW_TRAVIS_COVERAGE" == "1" ]; then
     # Output Python coverage data in a persistent place
     export COVERAGE_FILE=$ARROW_PYTHON_COVERAGE_FILE
-    coverage run --append -m pytest $PYARROW_PYTEST_FLAGS pyarrow/tests
+    python -m coverage run --append -m pytest $PYARROW_PYTEST_FLAGS pyarrow/tests
 else
     python -m pytest $PYARROW_PYTEST_FLAGS pyarrow/tests
 fi
diff --git a/cpp/src/arrow/flight/perf-server.cc b/cpp/src/arrow/flight/perf-server.cc
index d113117..43cede4 100644
--- a/cpp/src/arrow/flight/perf-server.cc
+++ b/cpp/src/arrow/flight/perf-server.cc
@@ -196,7 +196,7 @@ std::unique_ptr<arrow::flight::FlightPerfServer> g_server;
 
 void Shutdown(int signal) {
   if (g_server != nullptr) {
-    g_server->Shutdown();
+    ARROW_CHECK_OK(g_server->Shutdown());
   }
 }
 
diff --git a/cpp/src/arrow/flight/server.cc b/cpp/src/arrow/flight/server.cc
index f5f4698..849fbf0 100644
--- a/cpp/src/arrow/flight/server.cc
+++ b/cpp/src/arrow/flight/server.cc
@@ -584,9 +584,13 @@ Status FlightServerBase::Serve() {
 
 int FlightServerBase::GotSignal() const { return impl_->got_signal_; }
 
-void FlightServerBase::Shutdown() {
-  DCHECK(impl_->server_);
+Status FlightServerBase::Shutdown() {
+  auto server = impl_->server_.get();
+  if (!server) {
+    return Status::Invalid("Shutdown() on uninitialized FlightServerBase");
+  }
   impl_->server_->Shutdown();
+  return Status::OK();
 }
 
 Status FlightServerBase::ListFlights(const ServerCallContext& context,
diff --git a/cpp/src/arrow/flight/server.h b/cpp/src/arrow/flight/server.h
index 25656e6..d53eb43 100644
--- a/cpp/src/arrow/flight/server.h
+++ b/cpp/src/arrow/flight/server.h
@@ -142,7 +142,7 @@ class ARROW_FLIGHT_EXPORT FlightServerBase {
   /// thread while Serve() blocks.
   ///
   /// TODO(wesm): Shutdown with deadline
-  void Shutdown();
+  Status Shutdown();
 
   // Implement these methods to create your own server. The default
   // implementations will return a not-implemented result to the client
diff --git a/cpp/src/arrow/flight/test-util.cc b/cpp/src/arrow/flight/test-util.cc
index 4408801..e93e83f 100644
--- a/cpp/src/arrow/flight/test-util.cc
+++ b/cpp/src/arrow/flight/test-util.cc
@@ -142,7 +142,7 @@ Status InProcessTestServer::Start() {
 }
 
 void InProcessTestServer::Stop() {
-  server_->Shutdown();
+  ARROW_CHECK_OK(server_->Shutdown());
   thread_.join();
 }
 
diff --git a/python/examples/flight/server.py b/python/examples/flight/server.py
index 3b69972..f8a3563 100644
--- a/python/examples/flight/server.py
+++ b/python/examples/flight/server.py
@@ -122,8 +122,9 @@ def main():
             kwargs["tls_private_key"] = key_file.read()
 
     location = "{}://0.0.0.0:{}".format(scheme, args.port)
+    server.init(location, **kwargs)
     print("Serving on", location)
-    server.run(location, **kwargs)
+    server.run()
 
 
 if __name__ == '__main__':
diff --git a/python/pyarrow/_flight.pyx b/python/pyarrow/_flight.pyx
index a0d08f8..5bfc88a 100644
--- a/python/pyarrow/_flight.pyx
+++ b/python/pyarrow/_flight.pyx
@@ -1305,8 +1305,8 @@ cdef class FlightServerBase:
     cdef:
         unique_ptr[PyFlightServer] server
 
-    def run(self, location, auth_handler=None, tls_certificates=None):
-        """Start this server.
+    def init(self, location, auth_handler=None, tls_certificates=None):
+        """Initialize this server.
 
         Parameters
         ----------
@@ -1348,7 +1348,18 @@ cdef class FlightServerBase:
         self.server.reset(c_server)
         with nogil:
             check_status(c_server.Init(deref(c_options)))
-            check_status(c_server.ServeWithSignals())
+
+    def run(self):
+        """
+        Start serving.  This method only returns if shutdown() is called
+        or a signal a received.
+
+        You must have called init() first.
+        """
+        if self.server.get() == nullptr:
+            raise ValueError("run() on uninitialized FlightServerBase")
+        with nogil:
+            check_status(self.server.get().ServeWithSignals())
 
     def list_flights(self, context, criteria):
         raise NotImplementedError
@@ -1381,6 +1392,7 @@ cdef class FlightServerBase:
         # complete. Holding the GIL means Python-implemented Flight
         # methods will never get to run, so this will hang
         # indefinitely.
+        if self.server.get() == nullptr:
+            raise ValueError("shutdown() on uninitialized FlightServerBase")
         with nogil:
-            if self.server.get() != NULL:
-                self.server.get().Shutdown()
+            check_status(self.server.get().Shutdown())
diff --git a/python/pyarrow/includes/libarrow_flight.pxd b/python/pyarrow/includes/libarrow_flight.pxd
index 0639574..ed0b33e 100644
--- a/python/pyarrow/includes/libarrow_flight.pxd
+++ b/python/pyarrow/includes/libarrow_flight.pxd
@@ -281,7 +281,7 @@ cdef extern from "arrow/python/flight.h" namespace "arrow::py::flight" nogil:
 
         CStatus Init(CFlightServerOptions& options)
         CStatus ServeWithSignals() except *
-        void Shutdown()
+        CStatus Shutdown()
 
     cdef cppclass PyServerAuthHandler\
             " arrow::py::flight::PyServerAuthHandler"(CServerAuthHandler):
diff --git a/python/pyarrow/tests/test_flight.py b/python/pyarrow/tests/test_flight.py
index 3482c93..89b9a95 100644
--- a/python/pyarrow/tests/test_flight.py
+++ b/python/pyarrow/tests/test_flight.py
@@ -378,13 +378,14 @@ def flight_server(server_base, *args, **kwargs):
 
     ctor_kwargs = kwargs
     server_instance = server_base(*args, **ctor_kwargs)
+    # The server instance needs to be initialized before shutdown()
+    # can be called
+    server_instance.init(location,
+                         auth_handler=auth_handler,
+                         tls_certificates=tls_certificates)
 
     def _server_thread():
-        server_instance.run(
-            location,
-            auth_handler=auth_handler,
-            tls_certificates=tls_certificates,
-        )
+        server_instance.run()
 
     thread = threading.Thread(target=_server_thread, daemon=True)
     thread.start()
@@ -632,7 +633,7 @@ def test_location_invalid():
 
     server = ConstantFlightServer()
     with pytest.raises(pa.ArrowInvalid, match=".*Cannot parse URI:.*"):
-        server.run("%")
+        server.init("%")
 
 
 @pytest.mark.slow
@@ -654,9 +655,6 @@ def test_tls_fails():
             client.do_get(flight.Ticket(b'ints'))
 
 
-# ARROW-5930: Currently fails in Travis with SIGSEGV when tearing down
-# server
-@pytest.mark.slow
 @pytest.mark.requires_testing_data
 def test_tls_do_get():
     """Try a simple do_get call over TLS."""
@@ -673,9 +671,6 @@ def test_tls_do_get():
         assert data.equals(table)
 
 
-# ARROW-5930: Currently fails in Travis with SIGSEGV when tearing down
-# server
-@pytest.mark.slow
 @pytest.mark.requires_testing_data
 def test_tls_override_hostname():
     """Check that incorrectly overriding the hostname fails."""