Skip to content

Commit

Permalink
ARROW-4236: [java] Distinct plasma client create exceptions
Browse files Browse the repository at this point in the history
when ray puts an object in plasma store, there are 2 exceptions may be thrown, one is "An object with this ID already exists in the plasma store" and the other is "The plasma store ran out of memory and could not create this object",  distinct them rather than let them both be the same java class

@raulchen please help review

Author: yl187661 <[email protected]>
Author: lynn <[email protected]>

Closes apache#3306 from bibabolynn/dev_plasmaClientException and squashes the following commits:

3e512b6 <yl187661> add assert
88e1702 <yl187661> cpp lint
5586036 <yl187661> cpp lint
c4fe54f <yl187661> cpp lint
10ff110 <yl187661> plasmaClientTest catch duplicate object exception
acc7c06 <yl187661> indentation
5699eff <yl187661> blank line
4710d59 <yl187661> fix
f3d12a6 <lynn> distinct plasma client create exception
  • Loading branch information
yl187661 authored and pcmoritz committed Jan 24, 2019
1 parent 6540040 commit 3405cd4
Show file tree
Hide file tree
Showing 7 changed files with 88 additions and 23 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -104,15 +104,15 @@ JNIEXPORT jobject JNICALL Java_org_apache_arrow_plasma_PlasmaClientJNI_create(
std::shared_ptr<Buffer> data;
Status s = client->Create(oid, size, md, md_size, &data);
if (s.IsPlasmaObjectExists()) {
jclass Exception = env->FindClass("java/lang/Exception");
env->ThrowNew(Exception,
"An object with this ID already exists in the plasma store.");
jclass exceptionClass =
env->FindClass("org/apache/arrow/plasma/exceptions/DuplicateObjectException");
env->ThrowNew(exceptionClass, oid.hex().c_str());
return nullptr;
}
if (s.IsPlasmaStoreFull()) {
jclass Exception = env->FindClass("java/lang/Exception");
env->ThrowNew(Exception,
"The plasma store ran out of memory and could not create this object.");
jclass exceptionClass =
env->FindClass("org/apache/arrow/plasma/exceptions/PlasmaOutOfMemoryException");
env->ThrowNew(exceptionClass, "");
return nullptr;
}
ARROW_CHECK(s.ok());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,9 @@

import java.util.List;

import org.apache.arrow.plasma.exceptions.DuplicateObjectException;
import org.apache.arrow.plasma.exceptions.PlasmaOutOfMemoryException;

/**
* Object store interface, which provides the capabilities to put and get raw byte array, and serves.
*/
Expand All @@ -42,7 +45,8 @@ class ObjectStoreData {
* @param value The value to put in the object store.
* @param metadata encodes whatever metadata the user wishes to encode.
*/
void put(byte[] objectId, byte[] value, byte[] metadata);
void put(byte[] objectId, byte[] value, byte[] metadata)
throws DuplicateObjectException, PlasmaOutOfMemoryException;

/**
* Get a buffer from the PlasmaStore based on the <tt>objectId</tt>.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,10 @@

import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;

import org.apache.arrow.plasma.exceptions.DuplicateObjectException;
import org.apache.arrow.plasma.exceptions.PlasmaOutOfMemoryException;

/**
* The PlasmaClient is used to interface with a plasma store and manager.
Expand All @@ -45,18 +46,9 @@ public PlasmaClient(String storeSocketName, String managerSocketName, int releas
// interface methods --------------------

@Override
public void put(byte[] objectId, byte[] value, byte[] metadata) {
ByteBuffer buf = null;
try {
buf = PlasmaClientJNI.create(conn, objectId, value.length, metadata);
} catch (Exception e) {
System.err.println("ObjectId " + objectId + " error at PlasmaClient put");
e.printStackTrace();
}
if (buf == null) {
return;
}

public void put(byte[] objectId, byte[] value, byte[] metadata)
throws DuplicateObjectException, PlasmaOutOfMemoryException {
ByteBuffer buf = PlasmaClientJNI.create(conn, objectId, value.length, metadata);
buf.put(value);
PlasmaClientJNI.seal(conn, objectId);
PlasmaClientJNI.release(conn, objectId);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,9 @@

import java.nio.ByteBuffer;

import org.apache.arrow.plasma.exceptions.DuplicateObjectException;
import org.apache.arrow.plasma.exceptions.PlasmaOutOfMemoryException;

/**
* JNI static methods for PlasmaClient.
*/
Expand All @@ -28,7 +31,8 @@ public class PlasmaClientJNI {

public static native void disconnect(long conn);

public static native ByteBuffer create(long conn, byte[] objectId, int size, byte[] metadata);
public static native ByteBuffer create(long conn, byte[] objectId, int size, byte[] metadata)
throws DuplicateObjectException, PlasmaOutOfMemoryException;

public static native byte[] hash(long conn, byte[] objectId);

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* 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.
*/

package org.apache.arrow.plasma.exceptions;

public class DuplicateObjectException extends RuntimeException {

public DuplicateObjectException(String objectId) {
super("An object with ID " + objectId + " already exists in the plasma store.");
}

public DuplicateObjectException(String objectId, Throwable t) {
super("An object with ID " + objectId + " already exists in the plasma store.", t);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* 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.
*/

package org.apache.arrow.plasma.exceptions;

public class PlasmaOutOfMemoryException extends RuntimeException {

public PlasmaOutOfMemoryException() {
super("The plasma store ran out of memory.");
}

public PlasmaOutOfMemoryException(Throwable t) {
super("The plasma store ran out of memory.", t);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,9 @@
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;

import org.apache.arrow.plasma.exceptions.DuplicateObjectException;
import org.junit.Assert;

public class PlasmaClientTest {

private String storeSuffix = "/tmp/store";
Expand Down Expand Up @@ -142,8 +145,12 @@ public void doTest() {
assert Arrays.equals(values.get(0), value1);
assert Arrays.equals(values.get(1), value2);
System.out.println("Plasma java client get multi-object test success.");
pLink.put(id1, value1, null);
System.out.println("Plasma java client put same object twice exception test success.");
try {
pLink.put(id1, value1, null);
Assert.fail("Fail to throw DuplicateObjectException when put an object into plasma store twice.");
} catch (DuplicateObjectException e) {
System.out.println("Plasma java client put same object twice exception test success.");
}
byte[] id1Hash = pLink.hash(id1);
assert id1Hash != null;
System.out.println("Plasma java client hash test success.");
Expand Down

0 comments on commit 3405cd4

Please sign in to comment.