|
| 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