Skip to content

Commit ce90af9

Browse files
author
yeyuqiang
authored
Plasma Utils which provide contain, remove method (apache#55)
1 parent 808d96d commit ce90af9

File tree

5 files changed

+142
-59
lines changed

5 files changed

+142
-59
lines changed

core/src/main/java/org/apache/spark/io/pmem/MyPlasmaClient.java

Lines changed: 35 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,8 @@
1919
import org.apache.spark.SparkEnv;
2020
import org.apache.spark.internal.config.package$;
2121

22+
import java.nio.ByteBuffer;
23+
2224
/**
2325
* Upstream Plasma Client Wrapper.
2426
* To simplify the parameter passed to plasma client.
@@ -34,11 +36,39 @@ public MyPlasmaClient(String storeSocketName) {
3436
/**
3537
* Write to plasma store with an child object id
3638
*/
37-
public void write(String parentObjectId, int index, byte[] buffer) {
39+
public void writeChildObject(String parentObjectId, int index, byte[] buffer) {
3840
ChildObjectId objectId = new ChildObjectId(parentObjectId, index);
3941
put(objectId.toBytes(), buffer, null);
4042
}
4143

44+
public ByteBuffer getChildObject(String parentObjectId, int index) {
45+
ChildObjectId childObjectId = new ChildObjectId(parentObjectId, index);
46+
ByteBuffer buffer = getObjAsByteBuffer(childObjectId.toBytes(), 0, false);
47+
return buffer;
48+
}
49+
50+
/**
51+
* record the total child number
52+
*/
53+
public void recordChildObjectNumber(String parentObjectId, int num) {
54+
put(paddingParentObjectId(parentObjectId).getBytes(),
55+
ByteBuffer.allocate(4).putInt(num).array(), null);
56+
}
57+
58+
public int getChildObjectNumber(String parentObjectId) {
59+
ByteBuffer buffer = getObjAsByteBuffer(paddingParentObjectId(parentObjectId).getBytes(),
60+
0, false);
61+
if (buffer == null) {
62+
return -1;
63+
}
64+
return buffer.getInt();
65+
}
66+
67+
String paddingParentObjectId(String parentObjectId) {
68+
// Padding with - to prevent duplicate from child object id.
69+
return StringUtils.rightPad(parentObjectId, 20, "-");
70+
}
71+
4272
@Override
4373
public void finalize() {
4474
super.finalize();
@@ -84,8 +114,9 @@ public static MyPlasmaClient get() {
84114
}
85115

86116
public static void close() {
87-
client.finalize();
88-
client = null;
117+
if (client != null) {
118+
client.finalize();
119+
client = null;
120+
}
89121
}
90-
91122
}

core/src/main/java/org/apache/spark/io/pmem/PlasmaInputStream.java

Lines changed: 3 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -62,8 +62,7 @@ public PlasmaInputStream(String parentObjectId) {
6262
private boolean refill() {
6363
if (!buffer.hasRemaining()) {
6464
buffer.clear();
65-
ChildObjectId childObjectId = new ChildObjectId(parentObjectId, currChildObjectNumber++);
66-
ByteBuffer bufFromPlasma = client.getObjAsByteBuffer(childObjectId.toBytes(), 0, false);
65+
ByteBuffer bufFromPlasma = client.getChildObject(parentObjectId, currChildObjectNumber++);
6766
if (bufFromPlasma == null) {
6867
return false;
6968
}
@@ -74,11 +73,8 @@ private boolean refill() {
7473
}
7574

7675
@Override
77-
public synchronized int read() {
78-
if (!refill()) {
79-
return -1;
80-
}
81-
return buffer.get() & 0xFF;
76+
public int read() {
77+
throw new UnsupportedOperationException("The method is not implemented");
8278
}
8379

8480
@Override

core/src/main/java/org/apache/spark/io/pmem/PlasmaOutputStream.java

Lines changed: 4 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -62,11 +62,7 @@ public PlasmaOutputStream(String parentObjectId) {
6262

6363
@Override
6464
public void write(int b) {
65-
buffer.put((byte) b);
66-
if (!buffer.hasRemaining()) {
67-
writeToPlasma();
68-
buffer.clear();
69-
}
65+
throw new UnsupportedOperationException("The method is not implemented");
7066
}
7167

7268
@Override
@@ -95,13 +91,14 @@ public void write(byte[] b, int off, int len) {
9591
buffer.clear();
9692
currChildObjectNumber++;
9793
}
94+
client.recordChildObjectNumber(parentObjectId, currChildObjectNumber);
9895
}
9996

10097
private void writeToPlasma() {
10198
if (buffer.hasRemaining()) {
102-
client.write(parentObjectId, currChildObjectNumber, shrinkLastObjBuffer());
99+
client.writeChildObject(parentObjectId, currChildObjectNumber, shrinkLastObjBuffer());
103100
} else {
104-
client.write(parentObjectId, currChildObjectNumber, buffer.array());
101+
client.writeChildObject(parentObjectId, currChildObjectNumber, buffer.array());
105102
}
106103
}
107104

Lines changed: 39 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,39 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* http://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
15+
package org.apache.spark.io.pmem;
16+
17+
/**
18+
* Utility to operate object stored in plasma server
19+
*/
20+
public class PlasmaUtils {
21+
22+
private static MyPlasmaClient client = MyPlasmaClientHolder.get();
23+
24+
public static boolean contains(String objectId) {
25+
int num = client.getChildObjectNumber(objectId);
26+
return num > 0;
27+
}
28+
29+
public static void remove(String objectId) {
30+
int num = client.getChildObjectNumber(objectId);
31+
for (int i = 0; i < num; i++) {
32+
ChildObjectId childObjectId = new ChildObjectId(objectId, i);
33+
client.release(childObjectId.toBytes());
34+
client.delete(childObjectId.toBytes());
35+
}
36+
client.release(client.paddingParentObjectId(objectId).getBytes());
37+
client.delete(client.paddingParentObjectId(objectId).getBytes());
38+
}
39+
}

core/src/test/java/org/apache/spark/io/pmem/PlasmaOutputInputStreamSuite.java

Lines changed: 61 additions & 41 deletions
Original file line numberDiff line numberDiff line change
@@ -2,9 +2,7 @@
22

33
import org.apache.spark.SparkConf;
44
import org.apache.spark.SparkEnv;
5-
import org.junit.After;
6-
import org.junit.Before;
7-
import org.junit.Test;
5+
import org.junit.*;
86

97
import java.io.*;
108
import java.nio.ByteBuffer;
@@ -13,15 +11,16 @@
1311
import java.util.Arrays;
1412
import java.util.List;
1513
import java.util.Random;
14+
import java.util.concurrent.ExecutorService;
15+
import java.util.concurrent.Executors;
1616
import java.util.concurrent.TimeUnit;
1717
import java.util.regex.Pattern;
1818
import java.util.stream.Collectors;
1919
import java.util.stream.Stream;
2020

2121
import static org.junit.Assume.*;
2222
import static org.junit.Assert.*;
23-
import static org.mockito.Mockito.mock;
24-
import static org.mockito.Mockito.when;
23+
import static org.mockito.Mockito.*;
2524

2625
/**
2726
* Tests functionality of {@link PlasmaInputStream} and {@link PlasmaOutputStream}
@@ -30,15 +29,15 @@ public class PlasmaOutputInputStreamSuite {
3029

3130
private final static int DEFAULT_BUFFER_SIZE = 4096;
3231

33-
private final String plasmaStoreServer = "plasma-store-server";
34-
private final String plasmaStoreSocket = "/tmp/PlasmaOutputInputStreamSuite_socket_file";
35-
private final long memoryInBytes = 1000000000;
32+
private final static String plasmaStoreServer = "plasma-store-server";
33+
private final static String plasmaStoreSocket = "/tmp/PlasmaOutputInputStreamSuite_socket_file";
34+
private final static long memoryInBytes = 1000000000;
3635

37-
private Process process;
36+
private static Process process;
3837
private final Random random = new Random();
3938

40-
@Before
41-
public void setUp() {
39+
@BeforeClass
40+
public static void setUp() {
4241
boolean isAvailable = isPlasmaJavaAvailable();
4342
assumeTrue("Please make sure libplasma_java.so is installed" +
4443
" under LD_LIBRARY_PATH or java.library.path", isAvailable);
@@ -63,24 +62,6 @@ public void testWithNullData() throws IOException {
6362
pos.write(null);
6463
}
6564

66-
@Test
67-
public void testSingleWriteRead() {
68-
String blockId = "block_id_" + random.nextInt(10000000);
69-
byte[] bytesWrite = prepareByteBlockToWrite(1);
70-
PlasmaOutputStream pos = new PlasmaOutputStream(blockId);
71-
for (byte b : bytesWrite) {
72-
pos.write(b);
73-
}
74-
75-
byte[] bytesRead = new byte[bytesWrite.length];
76-
PlasmaInputStream pis = new PlasmaInputStream(blockId);
77-
for (int i = 0; i < bytesRead.length; i++) {
78-
bytesRead[i] = (byte) pis.read();
79-
}
80-
81-
assertArrayEquals(bytesWrite, bytesRead);
82-
}
83-
8465
@Test
8566
public void testBufferWriteRead() throws IOException {
8667
String blockId = "block_id_" + random.nextInt(10000000);
@@ -92,6 +73,9 @@ public void testBufferWriteRead() throws IOException {
9273
PlasmaInputStream pis = new PlasmaInputStream(blockId);
9374
pis.read(bytesRead);
9475
assertArrayEquals(bytesWrite, bytesRead);
76+
77+
PlasmaUtils.remove(blockId);
78+
assertFalse(PlasmaUtils.contains(blockId));
9579
}
9680

9781
@Test
@@ -108,8 +92,10 @@ public void testPartialBlockWriteRead() throws IOException {
10892
while ((len = pis.read(buffer)) != -1) {
10993
bytesRead.put(buffer, 0, len);
11094
}
111-
11295
assertArrayEquals(bytesWrite, bytesRead.array());
96+
97+
PlasmaUtils.remove(blockId);
98+
assertFalse(PlasmaUtils.contains(blockId));
11399
}
114100

115101
@Test
@@ -125,12 +111,46 @@ public void testMultiBlocksWriteRead() throws IOException {
125111
while (pis.read(buffer) != -1) {
126112
bytesRead.put(buffer);
127113
}
128-
129114
assertArrayEquals(bytesWrite, bytesRead.array());
115+
116+
PlasmaUtils.remove(blockId);
117+
assertFalse(PlasmaUtils.contains(blockId));
118+
}
119+
120+
@Test
121+
public void testMultiThreadReadWrite() throws InterruptedException {
122+
int processNum = Runtime.getRuntime().availableProcessors();
123+
ExecutorService threadPool = Executors.newFixedThreadPool(processNum);
124+
for (int i = 0; i < 10 * processNum; i++) {
125+
threadPool.submit(() -> {
126+
try {
127+
String blockId = "block_id_" + random.nextInt(10000000);
128+
byte[] bytesWrite = prepareByteBlockToWrite(5.7);
129+
PlasmaOutputStream pos = new PlasmaOutputStream(blockId);
130+
pos.write(bytesWrite);
131+
132+
ByteBuffer bytesRead = ByteBuffer.allocate(bytesWrite.length);
133+
PlasmaInputStream pis = new PlasmaInputStream(blockId);
134+
byte[] buffer = new byte[DEFAULT_BUFFER_SIZE];
135+
int len;
136+
while ((len = pis.read(buffer)) != -1) {
137+
bytesRead.put(buffer, 0, len);
138+
}
139+
assertArrayEquals(bytesWrite, bytesRead.array());
140+
141+
PlasmaUtils.remove(blockId);
142+
assertFalse(PlasmaUtils.contains(blockId));
143+
} catch (IOException ex) {
144+
ex.printStackTrace();
145+
}
146+
});
147+
}
148+
threadPool.shutdown();
149+
threadPool.awaitTermination(Long.MAX_VALUE, TimeUnit.MINUTES);
130150
}
131151

132-
@After
133-
public void tearDown() {
152+
@AfterClass
153+
public static void tearDown() {
134154
try {
135155
MyPlasmaClientHolder.close();
136156
stopPlasmaStore();
@@ -140,7 +160,7 @@ public void tearDown() {
140160
}
141161
}
142162

143-
public boolean isPlasmaJavaAvailable() {
163+
public static boolean isPlasmaJavaAvailable() {
144164
boolean available = true;
145165
try {
146166
System.loadLibrary("plasma_java");
@@ -150,22 +170,22 @@ public boolean isPlasmaJavaAvailable() {
150170
return available;
151171
}
152172

153-
private boolean isPlasmaStoreExist() {
173+
private static boolean isPlasmaStoreExist() {
154174
return Stream.of(System.getenv("PATH").split(Pattern.quote(File.pathSeparator)))
155175
.map(Paths::get)
156176
.anyMatch(path -> Files.exists(path.resolve(plasmaStoreServer)));
157177
}
158178

159-
private Process startProcess(String command) throws IOException {
179+
private static Process startProcess(String command) throws IOException {
160180
List<String> cmdList = Arrays.stream(command.split(" ")).collect(Collectors.toList());
161181
ProcessBuilder processBuilder = new ProcessBuilder(cmdList).inheritIO();
162182
Process process = processBuilder.start();
163183
return process;
164184
}
165185

166-
private boolean startPlasmaStore() throws IOException, InterruptedException {
186+
private static boolean startPlasmaStore() throws IOException, InterruptedException {
167187
String command = plasmaStoreServer + " -s " + plasmaStoreSocket + " -m " + memoryInBytes;
168-
this.process = startProcess(command);
188+
process = startProcess(command);
169189
int ticktock = 60;
170190
if (process != null) {
171191
while(!process.isAlive()) {
@@ -180,7 +200,7 @@ private boolean startPlasmaStore() throws IOException, InterruptedException {
180200
return false;
181201
}
182202

183-
private void stopPlasmaStore() throws InterruptedException {
203+
private static void stopPlasmaStore() throws InterruptedException {
184204
if (process != null && process.isAlive()) {
185205
process.destroyForcibly();
186206
int ticktock = 60;
@@ -194,7 +214,7 @@ private void stopPlasmaStore() throws InterruptedException {
194214
}
195215
}
196216

197-
private void deletePlasmaSocketFile() {
217+
private static void deletePlasmaSocketFile() {
198218
File socketFile = new File(plasmaStoreSocket);
199219
if (socketFile != null && socketFile.exists()) {
200220
socketFile.delete();
@@ -207,7 +227,7 @@ private byte[] prepareByteBlockToWrite(double numOfBlock) {
207227
return bytesToWrite;
208228
}
209229

210-
private void mockSparkEnv() {
230+
private static void mockSparkEnv() {
211231
SparkConf conf = new SparkConf();
212232
conf.set("spark.io.plasma.server.socket", plasmaStoreSocket);
213233
SparkEnv mockEnv = mock(SparkEnv.class);

0 commit comments

Comments
 (0)