Skip to content

Commit faf567f

Browse files
committed
Initial perf test server
Single stream perf: Transferred 500000000 records totaling 16000000000 bytes at 1159.463743 mb/s. 37993307.923483 record/s. 9278.041782 batch/s.
1 parent 7f1f2d1 commit faf567f

12 files changed

Lines changed: 471 additions & 19 deletions

File tree

java/flight/pom.xml

Lines changed: 20 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -115,6 +115,12 @@
115115
</extension>
116116
</extensions>
117117
<plugins>
118+
<plugin>
119+
<artifactId>maven-surefire-plugin</artifactId>
120+
<configuration>
121+
<enableAssertions>false</enableAssertions>
122+
</configuration>
123+
</plugin>
118124
<plugin>
119125
<groupId>org.apache.maven.plugins</groupId>
120126
<artifactId>maven-shade-plugin</artifactId>
@@ -153,21 +159,32 @@
153159
<version>0.5.0</version>
154160
<configuration>
155161
<protocArtifact>com.google.protobuf:protoc:${dep.protobuf.version}:exe:${os.detected.classifier}</protocArtifact>
156-
<protoSourceRoot>${basedir}/src/main/protobuf</protoSourceRoot>
157162
<clearOutputDirectory>false</clearOutputDirectory>
158163
<pluginId>grpc-java</pluginId>
159164
<pluginArtifact>io.grpc:protoc-gen-grpc-java:${dep.grpc.version}:exe:${os.detected.classifier}</pluginArtifact>
160-
<outputDirectory>${project.build.directory}/generated-sources/protobuf</outputDirectory>
161165
</configuration>
162166
<executions>
163167
<execution>
168+
<id>src</id>
169+
<configuration>
170+
<protoSourceRoot>${basedir}/src/main/protobuf</protoSourceRoot>
171+
<outputDirectory>${project.build.directory}/generated-sources/protobuf</outputDirectory>
172+
</configuration>
164173
<goals>
165174
<goal>compile</goal>
166175
<goal>compile-custom</goal>
167176
</goals>
177+
</execution>
178+
<execution>
179+
<id>test</id>
168180
<configuration>
169-
181+
<protoSourceRoot>${basedir}/src/test/protobuf</protoSourceRoot>
182+
<outputDirectory>${project.build.directory}/generated-test-sources//protobuf</outputDirectory>
170183
</configuration>
184+
<goals>
185+
<goal>compile</goal>
186+
<goal>compile-custom</goal>
187+
</goals>
171188
</execution>
172189
</executions>
173190
</plugin>

java/flight/src/main/java/org/apache/arrow/flight/ArrowMessage.java

Lines changed: 9 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -60,6 +60,8 @@
6060
*/
6161
class ArrowMessage {
6262

63+
public static final boolean FAST_PATH = true;
64+
6365
private static final int DESCRIPTOR_TAG = (FlightData.FLIGHT_DESCRIPTOR_FIELD_NUMBER << 3) | WireFormat.WIRETYPE_LENGTH_DELIMITED;
6466
private static final int BODY_TAG = (FlightData.DATA_BODY_FIELD_NUMBER << 3) | WireFormat.WIRETYPE_LENGTH_DELIMITED;
6567
private static final int HEADER_TAG = (FlightData.DATA_HEADER_FIELD_NUMBER << 3) | WireFormat.WIRETYPE_LENGTH_DELIMITED;
@@ -95,7 +97,7 @@ public ArrowMessage(FlightDescriptor descriptor, Schema schema) {
9597
FlatBufferBuilder builder = new FlatBufferBuilder();
9698
int schemaOffset = schema.getSchema(builder);
9799
ByteBuffer serializedMessage = MessageSerializer.serializeMessage(builder, MessageHeader.Schema, schemaOffset, 0);
98-
//serializedMessage.flip();
100+
serializedMessage = serializedMessage.slice();
99101
message = Message.getRootAsMessage(serializedMessage);
100102
bufs = ImmutableList.of();
101103
this.descriptor = descriptor;
@@ -105,6 +107,7 @@ public ArrowMessage(ArrowRecordBatch batch) {
105107
FlatBufferBuilder builder = new FlatBufferBuilder();
106108
int batchOffset = batch.writeTo(builder);
107109
ByteBuffer serializedMessage = MessageSerializer.serializeMessage(builder, MessageHeader.RecordBatch, batchOffset, batch.computeBodyLength());
110+
serializedMessage = serializedMessage.slice();
108111
this.message = Message.getRootAsMessage(serializedMessage);
109112
this.bufs = ImmutableList.copyOf(batch.getBuffers());
110113
this.descriptor = null;
@@ -182,7 +185,7 @@ private static ArrowMessage frame(BufferAllocator allocator, final InputStream s
182185
}
183186
int size = readRawVarint32(stream);
184187
body = allocator.buffer(size);
185-
ReadableBuffer readableBuffer = GetReadableBuffer.getReadableBuffer(stream);
188+
ReadableBuffer readableBuffer = FAST_PATH ? GetReadableBuffer.getReadableBuffer(stream) : null;
186189
if(readableBuffer != null) {
187190
readableBuffer.readBytes(body.nioBuffer(0, size));
188191
} else {
@@ -273,9 +276,11 @@ public DrainableByteBufInputStream(CompositeByteBuf buffer) {
273276
@Override
274277
public int drainTo(OutputStream target) throws IOException {
275278
int size = buf.readableBytes();
276-
if(!AddWritableBuffer.add(buf, target)) {
277-
buf.getBytes(0, target, buf.readableBytes());
279+
if(FAST_PATH && AddWritableBuffer.add(buf, target)) {
280+
return size;
278281
}
282+
283+
buf.getBytes(0, target, buf.readableBytes());
279284
return size;
280285
}
281286

java/flight/src/main/java/org/apache/arrow/flight/FlightDescriptor.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -81,7 +81,7 @@ Flight.FlightDescriptor toProtocol(){
8181
Flight.FlightDescriptor.Builder b = Flight.FlightDescriptor.newBuilder();
8282

8383
if(isCmd) {
84-
return b.setType(DescriptorType.PATH).setCmd(ByteString.copyFrom(cmd)).build();
84+
return b.setType(DescriptorType.CMD).setCmd(ByteString.copyFrom(cmd)).build();
8585
}
8686
return b.setType(DescriptorType.PATH).addAllPath(path).build();
8787
}

java/flight/src/main/java/org/apache/arrow/flight/FlightInfo.java

Lines changed: 1 addition & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,6 @@
1717
*/
1818
package org.apache.arrow.flight;
1919

20-
import java.nio.ByteBuffer;
2120
import java.util.List;
2221
import java.util.stream.Collectors;
2322

@@ -26,7 +25,6 @@
2625
import org.apache.arrow.vector.types.pojo.Schema;
2726

2827
import com.google.common.collect.ImmutableList;
29-
import com.google.flatbuffers.FlatBufferBuilder;
3028
import com.google.protobuf.ByteString;
3129

3230
public class FlightInfo {
@@ -74,15 +72,9 @@ public List<FlightEndpoint> getEndpoints() {
7472
}
7573

7674
FlightGetInfo toProtocol() {
77-
FlatBufferBuilder builder = new FlatBufferBuilder();
78-
int schemaOffset = schema.getSchema(builder);
79-
builder.finish(schemaOffset);
80-
ByteBuffer bb = builder.dataBuffer();
81-
byte[] bytes = new byte[bb.remaining()];
82-
bb.get(bytes);
8375
return Flight.FlightGetInfo.newBuilder()
8476
.addAllEndpoint(endpoints.stream().map(t -> t.toProtocol()).collect(Collectors.toList()))
85-
.setSchema(ByteString.copyFrom(bytes))
77+
.setSchema(ByteString.copyFrom(schema.toByteArray()))
8678
.setFlightDescriptor(descriptor.toProtocol())
8779
.setTotalBytes(FlightInfo.this.bytes)
8880
.setTotalRecords(records)

java/flight/src/main/java/org/apache/arrow/flight/grpc/AddWritableBuffer.java

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
*/
1818
package org.apache.arrow.flight.grpc;
1919

20+
import java.io.IOException;
2021
import java.io.OutputStream;
2122
import java.lang.reflect.Constructor;
2223
import java.lang.reflect.Field;
@@ -86,8 +87,14 @@ public class AddWritableBuffer {
8687
* @param buf The buffer to add.
8788
* @param stream The Candidate OutputStream to add to.
8889
* @return True if added. False if not possible.
90+
* @throws IOException
8991
*/
90-
public static boolean add(ByteBuf buf, OutputStream stream) {
92+
public static boolean add(ByteBuf buf, OutputStream stream) throws IOException {
93+
buf.readBytes(stream, buf.readableBytes());
94+
// if(true) {
95+
// return true;
96+
// }
97+
9198
if(bufChainOut == null) {
9299
return false;
93100
}

java/flight/src/main/java/org/apache/arrow/flight/grpc/GetReadableBuffer.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -54,6 +54,10 @@ public class GetReadableBuffer {
5454
}
5555

5656
public static ReadableBuffer getReadableBuffer(InputStream is) {
57+
// if(true) {
58+
// return null;
59+
// }
60+
5761
if(BUFFER_INPUT_STREAM == null || !is.getClass().equals(BUFFER_INPUT_STREAM)) {
5862
return null;
5963
}

java/flight/src/main/protobuf/flight.proto

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,7 @@ option java_package = "org.apache.arrow.flight.impl";
88
* Additionally, the a flight service and expose a set of actions that are available.
99
*/
1010
service FlightService {
11-
11+
1212
/*
1313
* Get a list of available streams given a particular criteria. Most flight services will expose one
1414
* or more streams that are readily available for retrieval. This api allows listing the streams
Lines changed: 188 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,188 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
* <p>
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
* <p>
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.arrow.flight.perf;
19+
20+
import java.io.IOException;
21+
import java.nio.ByteBuffer;
22+
import java.util.ArrayList;
23+
import java.util.List;
24+
import java.util.concurrent.Callable;
25+
26+
import org.apache.arrow.flight.Action;
27+
import org.apache.arrow.flight.ActionType;
28+
import org.apache.arrow.flight.Criteria;
29+
import org.apache.arrow.flight.FlightDescriptor;
30+
import org.apache.arrow.flight.FlightEndpoint;
31+
import org.apache.arrow.flight.FlightInfo;
32+
import org.apache.arrow.flight.FlightProducer;
33+
import org.apache.arrow.flight.FlightServer;
34+
import org.apache.arrow.flight.FlightStream;
35+
import org.apache.arrow.flight.Location;
36+
import org.apache.arrow.flight.Result;
37+
import org.apache.arrow.flight.Ticket;
38+
import org.apache.arrow.flight.example.ExampleFlightServer;
39+
import org.apache.arrow.flight.impl.Flight.PutResult;
40+
import org.apache.arrow.flight.perf.impl.PerfOuterClass.Perf;
41+
import org.apache.arrow.flight.perf.impl.PerfOuterClass.Token;
42+
import org.apache.arrow.memory.BufferAllocator;
43+
import org.apache.arrow.util.AutoCloseables;
44+
import org.apache.arrow.vector.BigIntVector;
45+
import org.apache.arrow.vector.VectorSchemaRoot;
46+
import org.apache.arrow.vector.types.Types.MinorType;
47+
import org.apache.arrow.vector.types.pojo.Field;
48+
import org.apache.arrow.vector.types.pojo.Schema;
49+
50+
import com.google.common.base.Preconditions;
51+
import com.google.common.collect.ImmutableList;
52+
import com.google.protobuf.InvalidProtocolBufferException;
53+
54+
public class PerformanceTestServer implements AutoCloseable {
55+
56+
private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExampleFlightServer.class);
57+
58+
private final FlightServer flightServer;
59+
private final Location location;
60+
private final BufferAllocator allocator;
61+
private final PerfProducer producer;
62+
63+
public PerformanceTestServer(BufferAllocator allocator, Location location) {
64+
this.allocator = allocator.newChildAllocator("perf-server", 0, Long.MAX_VALUE);
65+
this.location = location;
66+
this.producer = new PerfProducer();
67+
this.flightServer = new FlightServer(allocator, location.getPort(), producer);
68+
}
69+
70+
public Location getLocation() {
71+
return location;
72+
}
73+
74+
public void start() throws IOException {
75+
flightServer.start();
76+
}
77+
78+
@Override
79+
public void close() throws Exception {
80+
AutoCloseables.close(flightServer, allocator);
81+
}
82+
83+
private final class PerfProducer implements FlightProducer {
84+
85+
@Override
86+
public void getStream(Ticket ticket, ServerStreamListener listener) {
87+
VectorSchemaRoot root = null;
88+
try {
89+
Token token = Token.parseFrom(ticket.getBytes());
90+
Perf perf = token.getDefinition();
91+
Schema schema = Schema.deserialize(ByteBuffer.wrap(perf.getSchema().toByteArray()));
92+
root = VectorSchemaRoot.create(schema, allocator);
93+
BigIntVector a = (BigIntVector) root.getVector("a");
94+
BigIntVector b = (BigIntVector) root.getVector("b");
95+
BigIntVector c = (BigIntVector) root.getVector("c");
96+
BigIntVector d = (BigIntVector) root.getVector("d");
97+
listener.start(root);
98+
root.allocateNew();
99+
100+
int current = 0;
101+
int batches = 0;
102+
long i = token.getStart();
103+
while(i < token.getEnd()) {
104+
if(TestPerf.VALIDATE) {
105+
a.setSafe(current, i);
106+
}
107+
i++;
108+
current++;
109+
if (i % perf.getRecordsPerBatch() == 0) {
110+
root.setRowCount(current);
111+
listener.putNext();
112+
batches++;
113+
current = 0;
114+
root.allocateNew();
115+
}
116+
}
117+
118+
// send last partial batch.
119+
if(current != 0) {
120+
root.setRowCount(current);
121+
listener.putNext();
122+
batches++;
123+
}
124+
listener.completed();
125+
} catch (InvalidProtocolBufferException e) {
126+
throw new RuntimeException(e);
127+
} finally {
128+
try {
129+
AutoCloseables.close(root);
130+
} catch (Exception e) {
131+
throw new RuntimeException(e);
132+
}
133+
}
134+
}
135+
136+
@Override
137+
public void listFlights(Criteria criteria, StreamListener<FlightInfo> listener) {
138+
}
139+
140+
@Override
141+
public FlightInfo getFlightInfo(FlightDescriptor descriptor) {
142+
try {
143+
Preconditions.checkArgument(descriptor.isCommand());
144+
Perf exec = Perf.parseFrom(descriptor.getCommand());
145+
146+
final Schema pojoSchema = new Schema(ImmutableList.of(
147+
Field.nullable("a", MinorType.BIGINT.getType()),
148+
Field.nullable("b", MinorType.BIGINT.getType()),
149+
Field.nullable("c", MinorType.BIGINT.getType()),
150+
Field.nullable("d", MinorType.BIGINT.getType())
151+
));
152+
153+
Token token = Token.newBuilder().setDefinition(exec)
154+
.setStart(0)
155+
.setEnd(exec.getRecordsPerStream())
156+
.build();
157+
final Ticket ticket = new Ticket(token.toByteArray());
158+
159+
List<FlightEndpoint> endpoints = new ArrayList<>();
160+
for(int i =0; i < exec.getStreamCount(); i++) {
161+
endpoints.add(new FlightEndpoint(ticket, getLocation()));
162+
}
163+
164+
return new FlightInfo(pojoSchema, descriptor, endpoints, -1, exec.getRecordsPerStream() * exec.getStreamCount());
165+
} catch (InvalidProtocolBufferException e) {
166+
throw new RuntimeException(e);
167+
}
168+
}
169+
170+
@Override
171+
public Callable<PutResult> acceptPut(FlightStream flightStream) {
172+
return null;
173+
}
174+
175+
@Override
176+
public Result doAction(Action action) {
177+
return null;
178+
}
179+
180+
@Override
181+
public void listActions(StreamListener<ActionType> listener) {
182+
}
183+
184+
}
185+
}
186+
187+
188+

0 commit comments

Comments
 (0)