Skip to content

Commit efd70cd

Browse files
committed
refactor and cleanup done, some failing tests
1 parent 5f00fa4 commit efd70cd

File tree

8 files changed

+107
-427
lines changed

8 files changed

+107
-427
lines changed

presto-flight-shim/src/main/java/com/facebook/presto/flightshim/FlightShimPluginManager.java

Lines changed: 17 additions & 309 deletions
Large diffs are not rendered by default.

presto-flight-shim/src/main/java/com/facebook/presto/flightshim/FlightShimProducer.java

Lines changed: 44 additions & 64 deletions
Original file line numberDiff line numberDiff line change
@@ -14,28 +14,32 @@
1414
package com.facebook.presto.flightshim;
1515

1616
import com.facebook.airlift.json.JsonCodec;
17+
import com.facebook.airlift.json.JsonCodecFactory;
18+
import com.facebook.airlift.json.JsonObjectMapperProvider;
1719
import com.facebook.airlift.log.Logger;
1820
import com.facebook.plugin.arrow.ArrowBatchSource;
1921
import com.facebook.presto.Session;
22+
import com.facebook.presto.block.BlockJsonSerde;
2023
import com.facebook.presto.common.RuntimeStats;
24+
import com.facebook.presto.common.block.Block;
25+
import com.facebook.presto.common.block.BlockEncodingManager;
26+
import com.facebook.presto.common.type.RowType;
27+
import com.facebook.presto.common.type.Type;
2128
import com.facebook.presto.execution.QueryIdGenerator;
2229
import com.facebook.presto.metadata.Split;
2330
import com.facebook.presto.spi.ColumnHandle;
2431
import com.facebook.presto.spi.ColumnMetadata;
2532
import com.facebook.presto.spi.ConnectorId;
2633
import com.facebook.presto.spi.ConnectorPageSource;
27-
import com.facebook.presto.spi.ConnectorSession;
2834
import com.facebook.presto.spi.ConnectorSplit;
2935
import com.facebook.presto.spi.ConnectorTableHandle;
3036
import com.facebook.presto.spi.ConnectorTableLayoutHandle;
31-
import com.facebook.presto.spi.SplitContext;
3237
import com.facebook.presto.spi.TableHandle;
33-
import com.facebook.presto.spi.connector.Connector;
34-
import com.facebook.presto.spi.connector.ConnectorPageSourceProvider;
35-
import com.facebook.presto.spi.connector.ConnectorRecordSetProvider;
3638
import com.facebook.presto.spi.connector.ConnectorTransactionHandle;
3739
import com.facebook.presto.spi.security.Identity;
38-
import com.facebook.presto.split.RecordPageSourceProvider;
40+
import com.facebook.presto.split.PageSourceManager;
41+
import com.facebook.presto.type.TypeDeserializer;
42+
import com.google.common.collect.ImmutableMap;
3943
import org.apache.arrow.flight.BackpressureStrategy;
4044
import org.apache.arrow.flight.CallStatus;
4145
import org.apache.arrow.flight.NoOpFlightProducer;
@@ -49,13 +53,10 @@
4953
import java.util.Optional;
5054
import java.util.concurrent.ExecutorService;
5155

52-
import static com.facebook.airlift.json.JsonCodec.jsonCodec;
5356
import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager;
5457
import static com.facebook.presto.testing.TestingSession.DEFAULT_TIME_ZONE_KEY;
55-
import static com.google.common.base.Preconditions.checkState;
5658
import static com.google.common.collect.ImmutableList.toImmutableList;
5759
import static java.lang.String.format;
58-
import static java.util.Collections.unmodifiableList;
5960
import static java.util.Locale.ENGLISH;
6061
import static java.util.Objects.requireNonNull;
6162

@@ -64,20 +65,39 @@ public class FlightShimProducer
6465
implements Closeable
6566
{
6667
private static final Logger log = Logger.get(FlightShimProducer.class);
67-
private static final JsonCodec<FlightShimRequest> REQUEST_JSON_CODEC = jsonCodec(FlightShimRequest.class);
6868
private static final int CLIENT_POLL_TIME = 5000; // Backpressure poll time ms
6969
private final BufferAllocator allocator;
7070
private final FlightShimPluginManager pluginManager;
71+
private final PageSourceManager pageSourceManager;
7172
private final FlightShimConfig config;
7273
private final ExecutorService shimExecutor;
74+
private final JsonCodec<FlightShimRequest> requestCodec;
7375

7476
@Inject
75-
public FlightShimProducer(BufferAllocator allocator, FlightShimPluginManager pluginManager, FlightShimConfig config, @ForFlightShimServer ExecutorService shimExecutor)
77+
public FlightShimProducer(
78+
BufferAllocator allocator,
79+
FlightShimPluginManager pluginManager,
80+
FlightShimConfig config,
81+
@ForFlightShimServer ExecutorService shimExecutor,
82+
TypeDeserializer typeDeserializer,
83+
BlockEncodingManager blockEncodingManager,
84+
PageSourceManager pageSourceManager)
7685
{
7786
this.allocator = allocator.newChildAllocator("flight-shim", 0, Long.MAX_VALUE);
7887
this.pluginManager = requireNonNull(pluginManager, "pluginManager is null");
7988
this.config = requireNonNull(config, "config is null");
8089
this.shimExecutor = requireNonNull(shimExecutor, "shimExecutor is null");
90+
this.pageSourceManager = requireNonNull(pageSourceManager, "pageSourceManager is null");
91+
92+
JsonObjectMapperProvider provider = new JsonObjectMapperProvider();
93+
BlockJsonSerde.Deserializer blockDeserializer = new BlockJsonSerde.Deserializer(blockEncodingManager);
94+
provider.setJsonDeserializers(ImmutableMap.of(
95+
RowType.class, typeDeserializer,
96+
Type.class, typeDeserializer,
97+
Block.class, blockDeserializer));
98+
JsonCodecFactory jsonCodecFactory = new JsonCodecFactory(provider);
99+
100+
this.requestCodec = jsonCodecFactory.jsonCodec(FlightShimRequest.class);
81101
}
82102

83103
@Override
@@ -97,26 +117,18 @@ private void runGetStreamAsync(CallContext context, Ticket ticket, ServerStreamL
97117
final BackpressureStrategy backpressureStrategy = new BackpressureStrategy.CallbackBackpressureStrategy();
98118
backpressureStrategy.register(listener);
99119

100-
FlightShimRequest request = REQUEST_JSON_CODEC.fromJson(ticket.getBytes());
120+
FlightShimRequest request = requestCodec.fromJson(ticket.getBytes()); //REQUEST_JSON_CODEC.fromJson(ticket.getBytes());
101121
log.debug("Request for connector: %s", request.getConnectorId());
102122

103-
FlightShimPluginManager.ConnectorHolder connectorHolder = pluginManager.getConnector(request.getConnectorId());
104-
requireNonNull(connectorHolder, format("Requested connector not loaded: %s", request.getConnectorId()));
105-
106-
//Connector connector = connectorHolder.getConnector();
107-
ConnectorSplit connectorSplit = connectorHolder.getCodecSplit().fromJson(request.getSplitBytes());
108-
109-
List<? extends ColumnHandle> columnHandles = request.getColumnHandlesBytes().stream().map(
110-
columnHandleBytes -> connectorHolder.getCodecColumnHandle().fromJson(columnHandleBytes)
111-
).collect(toImmutableList());
123+
FlightShimPluginManager.ConnectorCodecs connectorCodecs = pluginManager.getConnectorCodecs(request.getConnectorId());
124+
requireNonNull(connectorCodecs, format("Requested connector not loaded: %s", request.getConnectorId()));
112125

113-
List<ColumnMetadata> columnsMetadata = columnHandles.stream()
114-
.map(connectorHolder::getColumnMetadata).collect(toImmutableList());
126+
ConnectorSplit connectorSplit = connectorCodecs.getCodecSplit().fromJson(request.getSplitBytes());
115127

116-
ConnectorTableHandle connectorTableHandle = connectorHolder.getCodecTableHandle().fromJson(request.getTableHandleBytes());
117-
ConnectorTransactionHandle transactionHandle = connectorHolder.getCodecTransactionHandle().fromJson(request.getTransactionHandleBytes());
128+
ConnectorTableHandle connectorTableHandle = connectorCodecs.getCodecTableHandle().fromJson(request.getTableHandleBytes());
129+
ConnectorTransactionHandle transactionHandle = connectorCodecs.getCodecTransactionHandle().fromJson(request.getTransactionHandleBytes());
118130
Optional<ConnectorTableLayoutHandle> connectorTableLayoutHandle =
119-
request.getTableLayoutHandleBytes().map(tableLayoutHandleBytes -> connectorHolder.getCodecTableLayoutHandle().fromJson(tableLayoutHandleBytes));
131+
request.getTableLayoutHandleBytes().map(tableLayoutHandleBytes -> connectorCodecs.getCodecTableLayoutHandle().fromJson(tableLayoutHandleBytes));
120132
TableHandle tableHandle = new TableHandle(new ConnectorId(request.getConnectorId()), connectorTableHandle, transactionHandle, connectorTableLayoutHandle);
121133

122134
// Create a dummy session to load the connector
@@ -129,23 +141,15 @@ private void runGetStreamAsync(CallContext context, Ticket ticket, ServerStreamL
129141
ConnectorId connectorId = new ConnectorId(request.getConnectorId());
130142
Split split = new Split(connectorId, transactionHandle, connectorSplit);
131143

132-
List<ColumnHandle> columnHandles2 = request.getColumnHandlesBytes().stream().map(
133-
columnHandleBytes -> connectorHolder.getCodecColumnHandle().fromJson(columnHandleBytes)
144+
List<ColumnHandle> columnHandles = request.getColumnHandlesBytes().stream().map(
145+
columnHandleBytes -> connectorCodecs.getCodecColumnHandle().fromJson(columnHandleBytes)
134146
).collect(toImmutableList());
135147

136-
ConnectorPageSource connectorPageSource = pluginManager.createPageSource(session, split, tableHandle, columnHandles2, new RuntimeStats());
137-
138-
ConnectorSession connectorSession = session.toConnectorSession(connectorId);
148+
List<ColumnMetadata> columnsMetadata = request.getOutputType().getFields().stream()
149+
.map(field -> ColumnMetadata.builder().setName(field.getName().orElse("$data$")).setType(field.getType()).build())
150+
.collect(toImmutableList());
139151

140-
/*ConnectorPageSourceProvider connectorPageSourceProvider = getConnectorPageSourceProvider(connector, connectorId);
141-
ConnectorPageSource connectorPageSource = connectorPageSourceProvider.createPageSource(
142-
transactionHandle,
143-
connectorSession,
144-
split,
145-
null,
146-
unmodifiableList(columnHandles),
147-
new SplitContext(false),
148-
new RuntimeStats());*/
152+
ConnectorPageSource connectorPageSource = pageSourceManager.createPageSource(session, split, tableHandle, columnHandles, new RuntimeStats());
149153

150154
try (ArrowBatchSource batchSource = new ArrowBatchSource(allocator, columnsMetadata, connectorPageSource, config.getMaxRowsPerBatch())) {
151155
listener.setUseZeroCopy(true);
@@ -177,30 +181,6 @@ private void runGetStreamAsync(CallContext context, Ticket ticket, ServerStreamL
177181
}
178182
}
179183

180-
private ConnectorPageSourceProvider getConnectorPageSourceProvider(Connector connector, ConnectorId connectorId)
181-
{
182-
ConnectorPageSourceProvider connectorPageSourceProvider = null;
183-
try {
184-
connectorPageSourceProvider = connector.getPageSourceProvider();
185-
requireNonNull(connectorPageSourceProvider, format("Connector %s returned a null page source provider", connectorId));
186-
}
187-
catch (UnsupportedOperationException ignored) {
188-
}
189-
190-
if (connectorPageSourceProvider == null) {
191-
ConnectorRecordSetProvider connectorRecordSetProvider = null;
192-
try {
193-
connectorRecordSetProvider = connector.getRecordSetProvider();
194-
requireNonNull(connectorRecordSetProvider, format("Connector %s returned a null record set provider", connectorId));
195-
}
196-
catch (UnsupportedOperationException ignored) {
197-
}
198-
checkState(connectorRecordSetProvider != null, "Connector %s has neither a PageSource or RecordSet provider", connectorId);
199-
connectorPageSourceProvider = new RecordPageSourceProvider(connectorRecordSetProvider);
200-
}
201-
return connectorPageSourceProvider;
202-
}
203-
204184
@Override
205185
public void close()
206186
{

presto-flight-shim/src/main/java/com/facebook/presto/flightshim/FlightShimRequest.java

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@
1313
*/
1414
package com.facebook.presto.flightshim;
1515

16+
import com.facebook.presto.common.type.RowType;
1617
import com.fasterxml.jackson.annotation.JsonCreator;
1718
import com.fasterxml.jackson.annotation.JsonProperty;
1819
import com.google.common.collect.ImmutableList;
@@ -25,6 +26,7 @@
2526
public class FlightShimRequest
2627
{
2728
private final String connectorId;
29+
private final RowType outputType;
2830
private final byte[] splitBytes;
2931
private final List<byte[]> columnHandlesBytes;
3032
private final byte[] tableHandleBytes;
@@ -34,13 +36,15 @@ public class FlightShimRequest
3436
@JsonCreator
3537
public FlightShimRequest(
3638
@JsonProperty("connectorId") String connectorId,
39+
@JsonProperty("outputType") RowType outputType,
3740
@JsonProperty("splitBytes") byte[] splitBytes,
3841
@JsonProperty("columnHandlesBytes") List<byte[]> columnHandlesBytes,
3942
@JsonProperty("tableHandleBytes") byte[] tableHandleBytes,
4043
@JsonProperty("tableLayoutHandleBytes") Optional<byte[]> tableLayoutHandleBytes,
4144
@JsonProperty("transactionHandleBytes") byte[] transactionHandleBytes)
4245
{
4346
this.connectorId = requireNonNull(connectorId, "connectorId is null");
47+
this.outputType = requireNonNull(outputType, "outputType is null");
4448
this.splitBytes = requireNonNull(splitBytes, "splitBytes is null");
4549
this.columnHandlesBytes = ImmutableList.copyOf(requireNonNull(columnHandlesBytes, "columnHandlesBytes is null"));
4650
this.tableHandleBytes = requireNonNull(tableHandleBytes, "tableHandleBytes is null");
@@ -54,6 +58,12 @@ public String getConnectorId()
5458
return connectorId;
5559
}
5660

61+
@JsonProperty
62+
public RowType getOutputType()
63+
{
64+
return outputType;
65+
}
66+
5767
@JsonProperty
5868
public byte[] getSplitBytes()
5969
{

presto-flight-shim/src/main/java/com/facebook/presto/flightshim/FlightShimServer.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -60,12 +60,12 @@ public static Injector initialize(Map<String, String> config, Module... extraMod
6060
return app.initialize();
6161
}
6262

63-
public static FlightServer start(Injector injector, FlightServer.Builder builder)
63+
public static FlightServer start(Injector injector, FlightServer.Builder builder, Map<String, Map<String, String>> additionalCatalogs)
6464
throws Exception
6565
{
6666
FlightShimPluginManager pluginManager = injector.getInstance(FlightShimPluginManager.class);
6767
pluginManager.loadPlugins();
68-
pluginManager.loadCatalogs();
68+
pluginManager.loadCatalogs(additionalCatalogs);
6969

7070
builder.allocator(injector.getInstance(BufferAllocator.class));
7171
FlightShimConfig config = injector.getInstance(FlightShimConfig.class);
@@ -126,7 +126,7 @@ public static void main(String[] args)
126126
Injector injector = initialize(config);
127127

128128
log.info("FlightShim server initializing");
129-
try (FlightServer server = start(injector, FlightServer.builder());
129+
try (FlightServer server = start(injector, FlightServer.builder(), ImmutableMap.of());
130130
FlightShimProducer producer = injector.getInstance(FlightShimProducer.class)) {
131131
log.info(format("======== FlightShim Server started on port: %s ========", server.getPort()));
132132
server.awaitTermination();

presto-flight-shim/src/test/java/com/facebook/presto/flightshim/AbstractTestFlightShimJdbcPlugins.java

Lines changed: 13 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
import com.facebook.presto.common.type.ArrayType;
1818
import com.facebook.presto.common.type.CharType;
1919
import com.facebook.presto.common.type.DecimalType;
20+
import com.facebook.presto.common.type.RowType;
2021
import com.facebook.presto.common.type.Type;
2122
import com.facebook.presto.common.type.VarcharType;
2223
import com.facebook.presto.plugin.jdbc.JdbcColumnHandle;
@@ -116,24 +117,6 @@ void testJdbcSplitWithAdditionalPredicate() throws Exception
116117
}
117118
}
118119

119-
@Test
120-
public void testSelectColumns()
121-
{
122-
assertSelectQueryFromColumns(ImmutableList.of(ORDERKEY_COLUMN, LINENUMBER_COLUMN));
123-
}
124-
125-
@Test
126-
public void testDateColumns()
127-
{
128-
assertSelectQueryFromColumns(ImmutableList.of(ORDERKEY_COLUMN, SHIPDATE_COLUMN));
129-
}
130-
131-
@Test
132-
public void testFloatingPointColumns()
133-
{
134-
assertSelectQueryFromColumns(ImmutableList.of(ORDERKEY_COLUMN, QUANTITY_COLUMN, EXTENDEDPRICE_COLUMN));
135-
}
136-
137120
private JdbcColumnHandle convertToJdbcColumnHandle(TpchColumnHandle columnHandle)
138121
{
139122
Type type = columnHandle.getType();
@@ -153,35 +136,42 @@ protected FlightShimRequest createTpchTableRequest(int partNumber, int totalPart
153136
String split = createJdbcSplit(getConnectorId(), "tpch", TPCH_TABLE);
154137
byte[] splitBytes = split.getBytes(StandardCharsets.UTF_8);
155138

139+
ImmutableList.Builder<RowType.Field> fieldBuilder = ImmutableList.builder();
156140
ImmutableList.Builder<byte[]> columnBuilder = ImmutableList.builder();
157141
for (TpchColumnHandle columnHandle : columnHandles) {
142+
fieldBuilder.add(new RowType.Field(Optional.of(columnHandle.getColumnName()), columnHandle.getType()));
158143
columnBuilder.add(COLUMN_HANDLE_JSON_CODEC.toJsonBytes(convertToJdbcColumnHandle(columnHandle)));
159144
}
145+
RowType outputType = RowType.from(fieldBuilder.build());
160146

161147
JdbcTableHandle tableHandle = new JdbcTableHandle(getConnectorId(), new SchemaTableName("tpch", TPCH_TABLE), getConnectorId(), "tpch", TPCH_TABLE);
162148
byte[] tableHandleBytes = TABLE_HANDLE_JSON_CODEC.toJsonBytes(tableHandle);
163149
byte[] transactionHandleBytes = TRANSACTION_HANDLE_JSON_CODEC.toJsonBytes(new JdbcTransactionHandle());
164150

165-
return new FlightShimRequest(getConnectorId(), splitBytes, columnBuilder.build(), tableHandleBytes, Optional.empty(), transactionHandleBytes);
151+
return new FlightShimRequest(getConnectorId(), outputType, splitBytes, columnBuilder.build(), tableHandleBytes, Optional.empty(), transactionHandleBytes);
166152
}
167153

168154
protected FlightShimRequest createTpchTableRequestWithTupleDomain() throws Exception
169155
{
170156
JdbcColumnHandle orderKeyHandle = convertToJdbcColumnHandle(getOrderKeyColumn());
171157
byte[] splitBytes = Files.readAllBytes(getResourceFile("split_tuple_domain.json").toPath());
172158

159+
ImmutableList.Builder<RowType.Field> fieldBuilder = ImmutableList.builder();
173160
List<JdbcColumnHandle> columnHandles = ImmutableList.of(orderKeyHandle);
174161
ImmutableList.Builder<byte[]> columnBuilder = ImmutableList.builder();
175162
for (JdbcColumnHandle columnHandle : columnHandles) {
163+
fieldBuilder.add(new RowType.Field(Optional.of(columnHandle.getColumnName()), columnHandle.getColumnType()));
176164
columnBuilder.add(COLUMN_HANDLE_JSON_CODEC.toJsonBytes(columnHandle));
177165
}
166+
RowType outputType = RowType.from(fieldBuilder.build());
178167

179168
JdbcTableHandle tableHandle = new JdbcTableHandle(getConnectorId(), new SchemaTableName("tpch", TPCH_TABLE), getConnectorId(), "tpch", TPCH_TABLE);
180169
byte[] tableHandleBytes = TABLE_HANDLE_JSON_CODEC.toJsonBytes(tableHandle);
181170
byte[] transactionHandleBytes = TRANSACTION_HANDLE_JSON_CODEC.toJsonBytes(new JdbcTransactionHandle());
182171

183172
return new FlightShimRequest(
184173
getConnectorId(),
174+
outputType,
185175
splitBytes,
186176
columnBuilder.build(),
187177
tableHandleBytes,
@@ -196,18 +186,22 @@ protected FlightShimRequest createTpchTableRequestWithAdditionalPredicate()
196186
JdbcColumnHandle orderKeyHandle = convertToJdbcColumnHandle(getOrderKeyColumn());
197187
byte[] splitBytes = Files.readAllBytes(getResourceFile("split_additional_predicate.json").toPath());
198188

189+
ImmutableList.Builder<RowType.Field> fieldBuilder = ImmutableList.builder();
199190
List<JdbcColumnHandle> columnHandles = ImmutableList.of(orderKeyHandle);
200191
ImmutableList.Builder<byte[]> columnBuilder = ImmutableList.builder();
201192
for (JdbcColumnHandle columnHandle : columnHandles) {
193+
fieldBuilder.add(new RowType.Field(Optional.of(columnHandle.getColumnName()), columnHandle.getColumnType()));
202194
columnBuilder.add(COLUMN_HANDLE_JSON_CODEC.toJsonBytes(columnHandle));
203195
}
196+
RowType outputType = RowType.from(fieldBuilder.build());
204197

205198
JdbcTableHandle tableHandle = new JdbcTableHandle(getConnectorId(), new SchemaTableName("tpch", TPCH_TABLE), getConnectorId(), "tpch", TPCH_TABLE);
206199
byte[] tableHandleBytes = TABLE_HANDLE_JSON_CODEC.toJsonBytes(tableHandle);
207200
byte[] transactionHandleBytes = TRANSACTION_HANDLE_JSON_CODEC.toJsonBytes(new JdbcTransactionHandle());
208201

209202
return new FlightShimRequest(
210203
getConnectorId(),
204+
outputType,
211205
splitBytes,
212206
columnBuilder.build(),
213207
tableHandleBytes,

0 commit comments

Comments
 (0)