You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ro...@apache.org on 2018/08/25 01:48:28 UTC

[arrow] branch master updated: ARROW-3018: [Plasma] Remove Mersenne twister

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

robertnishihara 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 06e7a1b  ARROW-3018: [Plasma] Remove Mersenne twister
06e7a1b is described below

commit 06e7a1ba5b89e2a25a0f1ba27220e7bcf500737a
Author: Philipp Moritz <pc...@gmail.com>
AuthorDate: Fri Aug 24 18:48:18 2018 -0700

    ARROW-3018: [Plasma] Remove Mersenne twister
    
    This removes the C++ random ObjectID generator. Having a global random number generator is not a good idea and with an existing random number generator, it is very easy to write code like:
    
    ```
    std::string random_string = // Create random string of length ObjectID::size()
    auto object_id = ObjectID::from_binary(random_string);
    ```
    
    Author: Philipp Moritz <pc...@gmail.com>
    
    Closes #2400 from pcmoritz/fix-plasma-rng-seed and squashes the following commits:
    
    b859abe <Philipp Moritz> add back regression test
    2d5a820 <Philipp Moritz> fix
    0faec01 <Philipp Moritz> introduce size
    7eab510 <Philipp Moritz> fix random seed
    36a180b <Philipp Moritz> fix
    5cbea99 <Philipp Moritz> wtf python 2
    1648e91 <Philipp Moritz> fix
    abb7928 <Philipp Moritz> fix
    68ae941 <Philipp Moritz> lint
    240c203 <Philipp Moritz> add test-common.h
    2961c94 <Philipp Moritz> fix
    22e477e <Philipp Moritz> update
    0803b61 <Philipp Moritz> remove ObjectID::from_random
    2cb3c0f <Philipp Moritz> fix
    2b00706 <Philipp Moritz> fix
    15c628e <Philipp Moritz> add regression test
    1463340 <Philipp Moritz> fix comment
    b7f5167 <Philipp Moritz> use /dev/urandom
    ad4361d <Philipp Moritz> cast
    85c059c <Philipp Moritz> fix plasma rng seed
---
 cpp/src/plasma/common.cc                   | 18 -----------
 cpp/src/plasma/common.h                    |  2 +-
 cpp/src/plasma/test-common.h               | 37 +++++++++++++++++++++
 cpp/src/plasma/test/client_tests.cc        | 40 +++++++++++------------
 cpp/src/plasma/test/serialization_tests.cc | 52 +++++++++++++++---------------
 python/pyarrow/_plasma.pyx                 | 22 ++++++++++---
 python/pyarrow/tests/test_plasma.py        | 11 ++++++-
 7 files changed, 111 insertions(+), 71 deletions(-)

diff --git a/cpp/src/plasma/common.cc b/cpp/src/plasma/common.cc
index 6e368d3..0ca17cf 100644
--- a/cpp/src/plasma/common.cc
+++ b/cpp/src/plasma/common.cc
@@ -18,8 +18,6 @@
 #include "plasma/common.h"
 
 #include <limits>
-#include <mutex>
-#include <random>
 
 #include "plasma/plasma_generated.h"
 
@@ -29,22 +27,6 @@ namespace plasma {
 
 using arrow::Status;
 
-UniqueID UniqueID::from_random() {
-  UniqueID id;
-  uint8_t* data = id.mutable_data();
-  // NOTE(pcm): The right way to do this is to have one std::mt19937 per
-  // thread (using the thread_local keyword), but that's not supported on
-  // older versions of macOS (see https://stackoverflow.com/a/29929949)
-  static std::mutex mutex;
-  std::lock_guard<std::mutex> lock(mutex);
-  static std::mt19937 generator;
-  std::uniform_int_distribution<uint32_t> dist(0, std::numeric_limits<uint8_t>::max());
-  for (int i = 0; i < kUniqueIDSize; i++) {
-    data[i] = static_cast<uint8_t>(dist(generator));
-  }
-  return id;
-}
-
 UniqueID UniqueID::from_binary(const std::string& binary) {
   UniqueID id;
   std::memcpy(&id, binary.data(), sizeof(id));
diff --git a/cpp/src/plasma/common.h b/cpp/src/plasma/common.h
index a41bf72..a8cb931 100644
--- a/cpp/src/plasma/common.h
+++ b/cpp/src/plasma/common.h
@@ -39,7 +39,6 @@ constexpr int64_t kUniqueIDSize = 20;
 
 class ARROW_EXPORT UniqueID {
  public:
-  static UniqueID from_random();
   static UniqueID from_binary(const std::string& binary);
   bool operator==(const UniqueID& rhs) const;
   const uint8_t* data() const;
@@ -47,6 +46,7 @@ class ARROW_EXPORT UniqueID {
   std::string binary() const;
   std::string hex() const;
   size_t hash() const;
+  static int64_t size() { return kUniqueIDSize; }
 
  private:
   uint8_t id_[kUniqueIDSize];
diff --git a/cpp/src/plasma/test-common.h b/cpp/src/plasma/test-common.h
new file mode 100644
index 0000000..66b8c24
--- /dev/null
+++ b/cpp/src/plasma/test-common.h
@@ -0,0 +1,37 @@
+// 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.
+
+#ifndef PLASMA_TEST_COMMON_H
+#define PLASMA_TEST_COMMON_H
+
+#include "arrow/test-util.h"
+#include "gtest/gtest.h"
+
+#include "plasma/common.h"
+
+namespace plasma {
+
+ObjectID random_object_id() {
+  static uint32_t random_seed = 0;
+  ObjectID result;
+  arrow::random_bytes(kUniqueIDSize, random_seed++, result.mutable_data());
+  return result;
+}
+
+}  // namespace plasma
+
+#endif  // PLASMA_TEST_COMMON_H
diff --git a/cpp/src/plasma/test/client_tests.cc b/cpp/src/plasma/test/client_tests.cc
index 4469c60..a1d43dc 100644
--- a/cpp/src/plasma/test/client_tests.cc
+++ b/cpp/src/plasma/test/client_tests.cc
@@ -26,15 +26,13 @@
 #include <random>
 #include <thread>
 
-#include "arrow/test-util.h"
+#include "plasma/test-common.h"
 
 #include "plasma/client.h"
 #include "plasma/common.h"
 #include "plasma/plasma.h"
 #include "plasma/protocol.h"
 
-#include "gtest/gtest.h"
-
 namespace plasma {
 
 std::string test_executable;  // NOLINT
@@ -106,7 +104,7 @@ TEST_F(TestPlasmaStore, NewSubscriberTest) {
   ARROW_CHECK_OK(local_client.Connect(store_socket_name_, ""));
   ARROW_CHECK_OK(local_client2.Connect(store_socket_name_, ""));
 
-  ObjectID object_id = ObjectID::from_random();
+  ObjectID object_id = random_object_id();
 
   // Test for the object being in local Plasma store.
   // First create object.
@@ -123,7 +121,7 @@ TEST_F(TestPlasmaStore, NewSubscriberTest) {
   ARROW_CHECK_OK(local_client2.Subscribe(&fd));
   ASSERT_GT(fd, 0);
 
-  ObjectID object_id2 = ObjectID::from_random();
+  ObjectID object_id2 = random_object_id();
   int64_t data_size2 = 0;
   int64_t metadata_size2 = 0;
   ARROW_CHECK_OK(
@@ -147,7 +145,7 @@ TEST_F(TestPlasmaStore, NewSubscriberTest) {
 }
 
 TEST_F(TestPlasmaStore, SealErrorsTest) {
-  ObjectID object_id = ObjectID::from_random();
+  ObjectID object_id = random_object_id();
 
   Status result = client_.Seal(object_id);
   ASSERT_TRUE(result.IsPlasmaObjectNonexistent());
@@ -162,7 +160,7 @@ TEST_F(TestPlasmaStore, SealErrorsTest) {
 }
 
 TEST_F(TestPlasmaStore, DeleteTest) {
-  ObjectID object_id = ObjectID::from_random();
+  ObjectID object_id = random_object_id();
 
   // Test for deleting non-existance object.
   Status result = client_.Delete(object_id);
@@ -192,8 +190,8 @@ TEST_F(TestPlasmaStore, DeleteTest) {
 }
 
 TEST_F(TestPlasmaStore, DeleteObjectsTest) {
-  ObjectID object_id1 = ObjectID::from_random();
-  ObjectID object_id2 = ObjectID::from_random();
+  ObjectID object_id1 = random_object_id();
+  ObjectID object_id2 = random_object_id();
 
   // Test for deleting non-existance object.
   Status result = client_.Delete(std::vector<ObjectID>{object_id1, object_id2});
@@ -242,7 +240,7 @@ TEST_F(TestPlasmaStore, DeleteObjectsTest) {
 }
 
 TEST_F(TestPlasmaStore, ContainsTest) {
-  ObjectID object_id = ObjectID::from_random();
+  ObjectID object_id = random_object_id();
 
   // Test for object non-existence.
   bool has_object;
@@ -263,7 +261,7 @@ TEST_F(TestPlasmaStore, ContainsTest) {
 TEST_F(TestPlasmaStore, GetTest) {
   std::vector<ObjectBuffer> object_buffers;
 
-  ObjectID object_id = ObjectID::from_random();
+  ObjectID object_id = random_object_id();
 
   // Test for object non-existence.
   ARROW_CHECK_OK(client_.Get({object_id}, 0, &object_buffers));
@@ -300,7 +298,7 @@ TEST_F(TestPlasmaStore, GetTest) {
 
 TEST_F(TestPlasmaStore, LegacyGetTest) {
   // Test for old non-releasing Get() variant
-  ObjectID object_id = ObjectID::from_random();
+  ObjectID object_id = random_object_id();
   {
     ObjectBuffer object_buffer;
 
@@ -328,8 +326,8 @@ TEST_F(TestPlasmaStore, LegacyGetTest) {
 }
 
 TEST_F(TestPlasmaStore, MultipleGetTest) {
-  ObjectID object_id1 = ObjectID::from_random();
-  ObjectID object_id2 = ObjectID::from_random();
+  ObjectID object_id1 = random_object_id();
+  ObjectID object_id2 = random_object_id();
   std::vector<ObjectID> object_ids = {object_id1, object_id2};
   std::vector<ObjectBuffer> object_buffers;
 
@@ -351,7 +349,7 @@ TEST_F(TestPlasmaStore, MultipleGetTest) {
 }
 
 TEST_F(TestPlasmaStore, AbortTest) {
-  ObjectID object_id = ObjectID::from_random();
+  ObjectID object_id = random_object_id();
   std::vector<ObjectBuffer> object_buffers;
 
   // Test for object non-existence.
@@ -397,7 +395,7 @@ TEST_F(TestPlasmaStore, AbortTest) {
 }
 
 TEST_F(TestPlasmaStore, MultipleClientTest) {
-  ObjectID object_id = ObjectID::from_random();
+  ObjectID object_id = random_object_id();
   std::vector<ObjectBuffer> object_buffers;
 
   // Test for object non-existence on the first client.
@@ -421,7 +419,7 @@ TEST_F(TestPlasmaStore, MultipleClientTest) {
 
   // Test that one client disconnecting does not interfere with the other.
   // First create object on the second client.
-  object_id = ObjectID::from_random();
+  object_id = random_object_id();
   ARROW_CHECK_OK(client2_.Create(object_id, data_size, metadata, metadata_size, &data));
   // Disconnect the first client.
   ARROW_CHECK_OK(client_.Disconnect());
@@ -438,7 +436,7 @@ TEST_F(TestPlasmaStore, ManyObjectTest) {
   // and leave the last third unsealed.
   std::vector<ObjectID> object_ids;
   for (int i = 0; i < 100; i++) {
-    ObjectID object_id = ObjectID::from_random();
+    ObjectID object_id = random_object_id();
     object_ids.push_back(object_id);
 
     // Test for object non-existence on the first client.
@@ -522,7 +520,7 @@ void AssertCudaRead(const std::shared_ptr<Buffer>& buffer,
 }  // namespace
 
 TEST_F(TestPlasmaStore, GetGPUTest) {
-  ObjectID object_id = ObjectID::from_random();
+  ObjectID object_id = random_object_id();
   std::vector<ObjectBuffer> object_buffers;
 
   // Test for object non-existence.
@@ -556,7 +554,7 @@ TEST_F(TestPlasmaStore, GetGPUTest) {
 }
 
 TEST_F(TestPlasmaStore, MultipleClientGPUTest) {
-  ObjectID object_id = ObjectID::from_random();
+  ObjectID object_id = random_object_id();
   std::vector<ObjectBuffer> object_buffers;
 
   // Test for object non-existence on the first client.
@@ -580,7 +578,7 @@ TEST_F(TestPlasmaStore, MultipleClientGPUTest) {
 
   // Test that one client disconnecting does not interfere with the other.
   // First create object on the second client.
-  object_id = ObjectID::from_random();
+  object_id = random_object_id();
   ARROW_CHECK_OK(
       client2_.Create(object_id, data_size, metadata, metadata_size, &data, 1));
   // Disconnect the first client.
diff --git a/cpp/src/plasma/test/serialization_tests.cc b/cpp/src/plasma/test/serialization_tests.cc
index 15df7de..d910334 100644
--- a/cpp/src/plasma/test/serialization_tests.cc
+++ b/cpp/src/plasma/test/serialization_tests.cc
@@ -18,13 +18,13 @@
 #include <sys/types.h>
 #include <unistd.h>
 
+#include "plasma/test-common.h"
+
 #include "plasma/common.h"
 #include "plasma/io.h"
 #include "plasma/plasma.h"
 #include "plasma/protocol.h"
 
-#include "gtest/gtest.h"
-
 namespace fb = plasma::flatbuf;
 
 namespace plasma {
@@ -76,7 +76,7 @@ PlasmaObject random_plasma_object(void) {
 
 TEST(PlasmaSerialization, CreateRequest) {
   int fd = create_temp_file();
-  ObjectID object_id1 = ObjectID::from_random();
+  ObjectID object_id1 = random_object_id();
   int64_t data_size1 = 42;
   int64_t metadata_size1 = 11;
   int device_num1 = 0;
@@ -99,7 +99,7 @@ TEST(PlasmaSerialization, CreateRequest) {
 
 TEST(PlasmaSerialization, CreateReply) {
   int fd = create_temp_file();
-  ObjectID object_id1 = ObjectID::from_random();
+  ObjectID object_id1 = random_object_id();
   PlasmaObject object1 = random_plasma_object();
   int64_t mmap_size1 = 1000000;
   ARROW_CHECK_OK(SendCreateReply(fd, object_id1, &object1, PlasmaError::OK, mmap_size1));
@@ -120,7 +120,7 @@ TEST(PlasmaSerialization, CreateReply) {
 
 TEST(PlasmaSerialization, SealRequest) {
   int fd = create_temp_file();
-  ObjectID object_id1 = ObjectID::from_random();
+  ObjectID object_id1 = random_object_id();
   unsigned char digest1[kDigestSize];
   memset(&digest1[0], 7, kDigestSize);
   ARROW_CHECK_OK(SendSealRequest(fd, object_id1, &digest1[0]));
@@ -135,7 +135,7 @@ TEST(PlasmaSerialization, SealRequest) {
 
 TEST(PlasmaSerialization, SealReply) {
   int fd = create_temp_file();
-  ObjectID object_id1 = ObjectID::from_random();
+  ObjectID object_id1 = random_object_id();
   ARROW_CHECK_OK(SendSealReply(fd, object_id1, PlasmaError::ObjectExists));
   std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaSealReply);
   ObjectID object_id2;
@@ -148,8 +148,8 @@ TEST(PlasmaSerialization, SealReply) {
 TEST(PlasmaSerialization, GetRequest) {
   int fd = create_temp_file();
   ObjectID object_ids[2];
-  object_ids[0] = ObjectID::from_random();
-  object_ids[1] = ObjectID::from_random();
+  object_ids[0] = random_object_id();
+  object_ids[1] = random_object_id();
   int64_t timeout_ms = 1234;
   ARROW_CHECK_OK(SendGetRequest(fd, object_ids, 2, timeout_ms));
   std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaGetRequest);
@@ -166,8 +166,8 @@ TEST(PlasmaSerialization, GetRequest) {
 TEST(PlasmaSerialization, GetReply) {
   int fd = create_temp_file();
   ObjectID object_ids[2];
-  object_ids[0] = ObjectID::from_random();
-  object_ids[1] = ObjectID::from_random();
+  object_ids[0] = random_object_id();
+  object_ids[1] = random_object_id();
   std::unordered_map<ObjectID, PlasmaObject> plasma_objects;
   plasma_objects[object_ids[0]] = random_plasma_object();
   plasma_objects[object_ids[1]] = random_plasma_object();
@@ -200,7 +200,7 @@ TEST(PlasmaSerialization, GetReply) {
 
 TEST(PlasmaSerialization, ReleaseRequest) {
   int fd = create_temp_file();
-  ObjectID object_id1 = ObjectID::from_random();
+  ObjectID object_id1 = random_object_id();
   ARROW_CHECK_OK(SendReleaseRequest(fd, object_id1));
   std::vector<uint8_t> data =
       read_message_from_file(fd, MessageType::PlasmaReleaseRequest);
@@ -212,7 +212,7 @@ TEST(PlasmaSerialization, ReleaseRequest) {
 
 TEST(PlasmaSerialization, ReleaseReply) {
   int fd = create_temp_file();
-  ObjectID object_id1 = ObjectID::from_random();
+  ObjectID object_id1 = random_object_id();
   ARROW_CHECK_OK(SendReleaseReply(fd, object_id1, PlasmaError::ObjectExists));
   std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaReleaseReply);
   ObjectID object_id2;
@@ -224,7 +224,7 @@ TEST(PlasmaSerialization, ReleaseReply) {
 
 TEST(PlasmaSerialization, DeleteRequest) {
   int fd = create_temp_file();
-  ObjectID object_id1 = ObjectID::from_random();
+  ObjectID object_id1 = random_object_id();
   ARROW_CHECK_OK(SendDeleteRequest(fd, std::vector<ObjectID>{object_id1}));
   std::vector<uint8_t> data =
       read_message_from_file(fd, MessageType::PlasmaDeleteRequest);
@@ -237,7 +237,7 @@ TEST(PlasmaSerialization, DeleteRequest) {
 
 TEST(PlasmaSerialization, DeleteReply) {
   int fd = create_temp_file();
-  ObjectID object_id1 = ObjectID::from_random();
+  ObjectID object_id1 = random_object_id();
   PlasmaError error1 = PlasmaError::ObjectExists;
   ARROW_CHECK_OK(SendDeleteReply(fd, std::vector<ObjectID>{object_id1},
                                  std::vector<PlasmaError>{error1}));
@@ -257,8 +257,8 @@ TEST(PlasmaSerialization, StatusRequest) {
   int fd = create_temp_file();
   constexpr int64_t num_objects = 2;
   ObjectID object_ids[num_objects];
-  object_ids[0] = ObjectID::from_random();
-  object_ids[1] = ObjectID::from_random();
+  object_ids[0] = random_object_id();
+  object_ids[1] = random_object_id();
   ARROW_CHECK_OK(SendStatusRequest(fd, object_ids, num_objects));
   std::vector<uint8_t> data =
       read_message_from_file(fd, MessageType::PlasmaStatusRequest);
@@ -273,8 +273,8 @@ TEST(PlasmaSerialization, StatusRequest) {
 TEST(PlasmaSerialization, StatusReply) {
   int fd = create_temp_file();
   ObjectID object_ids[2];
-  object_ids[0] = ObjectID::from_random();
-  object_ids[1] = ObjectID::from_random();
+  object_ids[0] = random_object_id();
+  object_ids[1] = random_object_id();
   int object_statuses[2] = {42, 43};
   ARROW_CHECK_OK(SendStatusReply(fd, object_ids, object_statuses, 2));
   std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaStatusReply);
@@ -316,8 +316,8 @@ TEST(PlasmaSerialization, EvictReply) {
 TEST(PlasmaSerialization, FetchRequest) {
   int fd = create_temp_file();
   ObjectID object_ids[2];
-  object_ids[0] = ObjectID::from_random();
-  object_ids[1] = ObjectID::from_random();
+  object_ids[0] = random_object_id();
+  object_ids[1] = random_object_id();
   ARROW_CHECK_OK(SendFetchRequest(fd, object_ids, 2));
   std::vector<uint8_t> data = read_message_from_file(fd, MessageType::PlasmaFetchRequest);
   std::vector<ObjectID> object_ids_read;
@@ -331,9 +331,9 @@ TEST(PlasmaSerialization, WaitRequest) {
   int fd = create_temp_file();
   const int num_objects_in = 2;
   ObjectRequest object_requests_in[num_objects_in] = {
-      ObjectRequest({ObjectID::from_random(), ObjectRequestType::PLASMA_QUERY_ANYWHERE,
+      ObjectRequest({random_object_id(), ObjectRequestType::PLASMA_QUERY_ANYWHERE,
                      ObjectLocation::Local}),
-      ObjectRequest({ObjectID::from_random(), ObjectRequestType::PLASMA_QUERY_LOCAL,
+      ObjectRequest({random_object_id(), ObjectRequestType::PLASMA_QUERY_LOCAL,
                      ObjectLocation::Local})};
   const int num_ready_objects_in = 1;
   int64_t timeout_ms = 1000;
@@ -365,10 +365,10 @@ TEST(PlasmaSerialization, WaitReply) {
   const int num_objects_in = 2;
   /* Create a map with two ObjectRequests in it. */
   ObjectRequestMap objects_in(num_objects_in);
-  ObjectID id1 = ObjectID::from_random();
+  ObjectID id1 = random_object_id();
   objects_in[id1] =
       ObjectRequest({id1, ObjectRequestType::PLASMA_QUERY_LOCAL, ObjectLocation::Local});
-  ObjectID id2 = ObjectID::from_random();
+  ObjectID id2 = random_object_id();
   objects_in[id2] = ObjectRequest(
       {id2, ObjectRequestType::PLASMA_QUERY_LOCAL, ObjectLocation::Nonexistent});
 
@@ -393,7 +393,7 @@ TEST(PlasmaSerialization, WaitReply) {
 
 TEST(PlasmaSerialization, DataRequest) {
   int fd = create_temp_file();
-  ObjectID object_id1 = ObjectID::from_random();
+  ObjectID object_id1 = random_object_id();
   const char* address1 = "address1";
   int port1 = 12345;
   ARROW_CHECK_OK(SendDataRequest(fd, object_id1, address1, port1));
@@ -413,7 +413,7 @@ TEST(PlasmaSerialization, DataRequest) {
 
 TEST(PlasmaSerialization, DataReply) {
   int fd = create_temp_file();
-  ObjectID object_id1 = ObjectID::from_random();
+  ObjectID object_id1 = random_object_id();
   int64_t object_size1 = 146;
   int64_t metadata_size1 = 198;
   ARROW_CHECK_OK(SendDataReply(fd, object_id1, object_size1, metadata_size1));
diff --git a/python/pyarrow/_plasma.pyx b/python/pyarrow/_plasma.pyx
index b638c38..86a6232 100644
--- a/python/pyarrow/_plasma.pyx
+++ b/python/pyarrow/_plasma.pyx
@@ -28,6 +28,7 @@ from cpython.pycapsule cimport *
 
 import collections
 import pyarrow
+import random
 
 from pyarrow.lib cimport Buffer, NativeFile, check_status, pyarrow_wrap_buffer
 from pyarrow.includes.libarrow cimport (CBuffer, CMutableBuffer,
@@ -53,6 +54,9 @@ cdef extern from "plasma/common.h" nogil:
 
         c_string binary() const
 
+        @staticmethod
+        int64_t size()
+
     cdef struct CObjectRequest" plasma::ObjectRequest":
         CUniqueID object_id
         int type
@@ -136,7 +140,8 @@ cdef class ObjectID:
         CUniqueID data
 
     def __cinit__(self, object_id):
-        if not isinstance(object_id, bytes) or len(object_id) != 20:
+        if (not isinstance(object_id, bytes) or
+                len(object_id) != CUniqueID.size()):
             raise ValueError("Object ID must by 20 bytes,"
                              " is " + str(object_id))
         self.data = CUniqueID.from_binary(object_id)
@@ -169,8 +174,17 @@ cdef class ObjectID:
 
     @staticmethod
     def from_random():
-        cdef CUniqueID data = CUniqueID.from_random()
-        return ObjectID(data.binary())
+        """
+        Returns a randomly generated ObjectID.
+
+        Returns
+        -------
+        ObjectID
+            A randomly generated ObjectID.
+        """
+        random_id = bytes(bytearray(
+            random.getrandbits(8) for _ in range(CUniqueID.size())))
+        return ObjectID(random_id)
 
 
 cdef class ObjectNotAvailable:
@@ -627,7 +641,7 @@ cdef class PlasmaClient:
         int
             The metadata size of the object that was stored.
         """
-        cdef ObjectID object_id = ObjectID(20 * b"\0")
+        cdef ObjectID object_id = ObjectID(CUniqueID.size() * b"\0")
         cdef int64_t data_size
         cdef int64_t metadata_size
         with nogil:
diff --git a/python/pyarrow/tests/test_plasma.py b/python/pyarrow/tests/test_plasma.py
index 99a5d5b..7cb1c0e 100644
--- a/python/pyarrow/tests/test_plasma.py
+++ b/python/pyarrow/tests/test_plasma.py
@@ -23,6 +23,7 @@ import os
 import pytest
 import random
 import signal
+import subprocess
 import sys
 
 import numpy as np
@@ -278,7 +279,7 @@ class TestPlasmaClient(object):
             result = self.plasma_client.get(object_id)
             assert result == value
 
-            object_id = pa.plasma.ObjectID.from_random()
+            object_id = random_object_id()
             [result] = self.plasma_client.get([object_id], timeout_ms=0)
             assert result == pa.plasma.ObjectNotAvailable
 
@@ -790,3 +791,11 @@ def test_plasma_client_sharing():
         del plasma_client
         assert (buf == np.zeros(3)).all()
         del buf  # This segfaulted pre ARROW-2448.
+
+
+@pytest.mark.plasma
+def test_object_id_randomness():
+    cmd = "from pyarrow import plasma; print(plasma.ObjectID.from_random())"
+    first_object_id = subprocess.check_output(["python", "-c", cmd])
+    second_object_id = subprocess.check_output(["python", "-c", cmd])
+    assert first_object_id != second_object_id