From ba998dd1367f3b7a3a485b9eaf39b2718aa992d3 Mon Sep 17 00:00:00 2001 From: Licht-T Date: Thu, 23 Nov 2017 01:18:36 +0900 Subject: [PATCH 1/2] CLN: Remove pickle=True option for object serialization --- python/pyarrow/serialization.pxi | 14 ++------------ python/pyarrow/serialization.py | 7 +++++-- python/pyarrow/tests/test_serialization.py | 9 ++++++--- 3 files changed, 13 insertions(+), 17 deletions(-) diff --git a/python/pyarrow/serialization.pxi b/python/pyarrow/serialization.pxi index 6b7227797a8..3ee5c7d4e27 100644 --- a/python/pyarrow/serialization.pxi +++ b/python/pyarrow/serialization.pxi @@ -47,7 +47,6 @@ cdef class SerializationContext: cdef: object type_to_type_id object whitelisted_types - object types_to_pickle object custom_serializers object custom_deserializers @@ -55,11 +54,10 @@ cdef class SerializationContext: # Types with special serialization handlers self.type_to_type_id = dict() self.whitelisted_types = dict() - self.types_to_pickle = set() self.custom_serializers = dict() self.custom_deserializers = dict() - def register_type(self, type_, type_id, pickle=False, + def register_type(self, type_, type_id, custom_serializer=None, custom_deserializer=None): """EXPERIMENTAL: Add type to the list of types we can serialize. @@ -69,9 +67,6 @@ cdef class SerializationContext: The type that we can serialize. type_id : bytes A string of bytes used to identify the type. - pickle : bool - True if the serialization should be done with pickle. - False if it should be done efficiently with Arrow. custom_serializer : callable This argument is optional, but can be provided to serialize objects of the class in a particular way. @@ -81,8 +76,6 @@ cdef class SerializationContext: """ self.type_to_type_id[type_] = type_id self.whitelisted_types[type_id] = type_ - if pickle: - self.types_to_pickle.add(type_id) if custom_serializer is not None: self.custom_serializers[type_id] = custom_serializer self.custom_deserializers[type_id] = custom_deserializer @@ -102,9 +95,7 @@ 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} - elif type_id in self.custom_serializers: + if type_id in self.custom_serializers: serialized_obj = {"data": self.custom_serializers[type_id](obj)} else: if is_named_tuple(type_): @@ -125,7 +116,6 @@ cdef class SerializationContext: # The object was pickled, so unpickle it. obj = pickle.loads(serialized_obj["data"]) else: - assert type_id not in self.types_to_pickle if type_id not in self.whitelisted_types: msg = "Type ID " + str(type_id) + " not registered in " \ "deserialization callback" diff --git a/python/pyarrow/serialization.py b/python/pyarrow/serialization.py index 2b47513fd1c..a1b25dc90c7 100644 --- a/python/pyarrow/serialization.py +++ b/python/pyarrow/serialization.py @@ -17,6 +17,7 @@ from collections import OrderedDict, defaultdict import sys +import pickle import numpy as np @@ -67,9 +68,11 @@ def _deserialize_default_dict(data): serialization_context.register_type( type(lambda: 0), "function", - pickle=True) + custom_serializer=pickle.dumps, custom_deserializer=pickle.loads) - serialization_context.register_type(type, "type", pickle=True) + serialization_context.register_type(type, "type", + custom_serializer=pickle.dumps, + custom_deserializer=pickle.loads) # ---------------------------------------------------------------------- # Set up serialization for numpy with dtype object (primitive types are diff --git a/python/pyarrow/tests/test_serialization.py b/python/pyarrow/tests/test_serialization.py index b0c5bc49e6a..ed4fd9ae59a 100644 --- a/python/pyarrow/tests/test_serialization.py +++ b/python/pyarrow/tests/test_serialization.py @@ -23,6 +23,7 @@ import datetime import string import sys +import pickle import pyarrow as pa import numpy as np @@ -197,7 +198,9 @@ def make_serialization_context(): context.register_type(Baz, "Baz") context.register_type(Qux, "Quz") context.register_type(SubQux, "SubQux") - context.register_type(SubQuxPickle, "SubQuxPickle", pickle=True) + context.register_type(SubQuxPickle, "SubQuxPickle", + custom_serializer=pickle.dumps, + custom_deserializer=pickle.loads) context.register_type(Exception, "Exception") context.register_type(CustomError, "CustomError") context.register_type(Point, "Point") @@ -338,7 +341,7 @@ def deserialize_dummy_class(serialized_obj): return serialized_obj pa._default_serialization_context.register_type( - DummyClass, "DummyClass", pickle=False, + DummyClass, "DummyClass", custom_serializer=serialize_dummy_class, custom_deserializer=deserialize_dummy_class) @@ -357,7 +360,7 @@ def deserialize_buffer_class(serialized_obj): return serialized_obj pa._default_serialization_context.register_type( - BufferClass, "BufferClass", pickle=False, + BufferClass, "BufferClass", custom_serializer=serialize_buffer_class, custom_deserializer=deserialize_buffer_class) From 927f1542679e225cdb3e2ea75d7cebc39c729a7e Mon Sep 17 00:00:00 2001 From: Wes McKinney Date: Sun, 26 Nov 2017 13:58:16 -0500 Subject: [PATCH 2/2] Use cloudpickle for lambda serialization if available Change-Id: Id4423a228ae2388c3e3f75d5650f0f0126fa9cc8 --- python/pyarrow/serialization.py | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/python/pyarrow/serialization.py b/python/pyarrow/serialization.py index a1b25dc90c7..ab25b63d571 100644 --- a/python/pyarrow/serialization.py +++ b/python/pyarrow/serialization.py @@ -24,6 +24,11 @@ from pyarrow import serialize_pandas, deserialize_pandas from pyarrow.lib import _default_serialization_context +try: + import cloudpickle +except ImportError: + cloudpickle = pickle + def register_default_serialization_handlers(serialization_context): @@ -68,11 +73,12 @@ def _deserialize_default_dict(data): serialization_context.register_type( type(lambda: 0), "function", - custom_serializer=pickle.dumps, custom_deserializer=pickle.loads) + custom_serializer=cloudpickle.dumps, + custom_deserializer=cloudpickle.loads) serialization_context.register_type(type, "type", - custom_serializer=pickle.dumps, - custom_deserializer=pickle.loads) + custom_serializer=cloudpickle.dumps, + custom_deserializer=cloudpickle.loads) # ---------------------------------------------------------------------- # Set up serialization for numpy with dtype object (primitive types are