You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by pc...@apache.org on 2018/01/19 22:49:26 UTC

[arrow] branch master updated: ARROW-2011: [Python] Allow setting the pickler in the serialization context.

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

pcmoritz 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 e446084  ARROW-2011: [Python] Allow setting the pickler in the serialization context.
e446084 is described below

commit e4460847f3387c6c1a8bb77edd2aedc69e7250d3
Author: Robert Nishihara <ro...@gmail.com>
AuthorDate: Fri Jan 19 14:49:17 2018 -0800

    ARROW-2011: [Python] Allow setting the pickler in the serialization context.
    
    Author: Robert Nishihara <ro...@gmail.com>
    
    Closes #1493 from robertnishihara/cloudpickle and squashes the following commits:
    
    57fb46f [Robert Nishihara] Fix test (it didn't work without cloudpickle).
    a884bb4 [Robert Nishihara] Add test.
    14e1536 [Robert Nishihara] Allow setting the pickler in the serialization context.
---
 python/pyarrow/serialization.pxi           | 26 ++++++++++++++++++--
 python/pyarrow/tests/test_serialization.py | 39 ++++++++++++++++++++++++++++++
 2 files changed, 63 insertions(+), 2 deletions(-)

diff --git a/python/pyarrow/serialization.pxi b/python/pyarrow/serialization.pxi
index d95d582..e7a3990 100644
--- a/python/pyarrow/serialization.pxi
+++ b/python/pyarrow/serialization.pxi
@@ -50,6 +50,8 @@ cdef class SerializationContext:
         object types_to_pickle
         object custom_serializers
         object custom_deserializers
+        object pickle_serializer
+        object pickle_deserializer
 
     def __init__(self):
         # Types with special serialization handlers
@@ -58,6 +60,23 @@ cdef class SerializationContext:
         self.types_to_pickle = set()
         self.custom_serializers = dict()
         self.custom_deserializers = dict()
+        self.pickle_serializer = pickle.dumps
+        self.pickle_deserializer = pickle.loads
+
+    def set_pickle(self, serializer, deserializer):
+        """
+        Set the serializer and deserializer to use for objects that are to be
+        pickled.
+
+        Parameters
+        ----------
+        serializer : callable
+            The serializer to use (e.g., pickle.dumps or cloudpickle.dumps).
+        deserializer : callable
+            The deserializer to use (e.g., pickle.dumps or cloudpickle.dumps).
+        """
+        self.pickle_serializer = serializer
+        self.pickle_deserializer = deserializer
 
     def clone(self):
         """
@@ -72,6 +91,8 @@ cdef class SerializationContext:
         result.whitelisted_types = self.whitelisted_types.copy()
         result.custom_serializers = self.custom_serializers.copy()
         result.custom_deserializers = self.custom_deserializers.copy()
+        result.pickle_serializer = self.pickle_serializer
+        result.pickle_deserializer = self.pickle_deserializer
 
         return result
 
@@ -119,7 +140,8 @@ cdef class SerializationContext:
         # use the closest match to type(obj)
         type_id = self.type_to_type_id[type_]
         if type_id in self.types_to_pickle:
-            serialized_obj = {"data": pickle.dumps(obj), "pickle": True}
+            serialized_obj = {"data": self.pickle_serializer(obj),
+                              "pickle": True}
         elif type_id in self.custom_serializers:
             serialized_obj = {"data": self.custom_serializers[type_id](obj)}
         else:
@@ -139,7 +161,7 @@ cdef class SerializationContext:
 
         if "pickle" in serialized_obj:
             # The object was pickled, so unpickle it.
-            obj = pickle.loads(serialized_obj["data"])
+            obj = self.pickle_deserializer(serialized_obj["data"])
         else:
             assert type_id not in self.types_to_pickle
             if type_id not in self.whitelisted_types:
diff --git a/python/pyarrow/tests/test_serialization.py b/python/pyarrow/tests/test_serialization.py
index 6116556..e4681e3 100644
--- a/python/pyarrow/tests/test_serialization.py
+++ b/python/pyarrow/tests/test_serialization.py
@@ -555,3 +555,42 @@ def test_deserialize_buffer_in_different_process():
     dir_path = os.path.dirname(os.path.realpath(__file__))
     python_file = os.path.join(dir_path, 'deserialize_buffer.py')
     subprocess.check_call(['python', python_file, f.name])
+
+
+def test_set_pickle():
+    # Use a custom type to trigger pickling.
+    class Foo(object):
+        pass
+
+    context = pa.SerializationContext()
+    context.register_type(Foo, 'Foo', pickle=True)
+
+    test_object = Foo()
+
+    # Define a custom serializer and deserializer to use in place of pickle.
+
+    def dumps1(obj):
+        return b'custom'
+
+    def loads1(serialized_obj):
+        return serialized_obj + b' serialization 1'
+
+    # Test that setting a custom pickler changes the behavior.
+    context.set_pickle(dumps1, loads1)
+    serialized = pa.serialize(test_object, context=context).to_buffer()
+    deserialized = pa.deserialize(serialized.to_pybytes(), context=context)
+    assert deserialized == b'custom serialization 1'
+
+    # Define another custom serializer and deserializer.
+
+    def dumps2(obj):
+        return b'custom'
+
+    def loads2(serialized_obj):
+        return serialized_obj + b' serialization 2'
+
+    # Test that setting another custom pickler changes the behavior again.
+    context.set_pickle(dumps2, loads2)
+    serialized = pa.serialize(test_object, context=context).to_buffer()
+    deserialized = pa.deserialize(serialized.to_pybytes(), context=context)
+    assert deserialized == b'custom serialization 2'

-- 
To stop receiving notification emails like this one, please contact
['"commits@arrow.apache.org" <co...@arrow.apache.org>'].