Skip to content

Commit b49b416

Browse files
committed
Make all in-tree connectors utilize new SPI method in ConnectorMetadata
1 parent 7a20948 commit b49b416

File tree

34 files changed

+143
-93
lines changed

34 files changed

+143
-93
lines changed

presto-accumulo/src/main/java/com/facebook/presto/accumulo/AccumuloMetadata.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -259,15 +259,15 @@ public ConnectorTableHandle getTableHandle(ConnectorSession session, SchemaTable
259259
}
260260

261261
@Override
262-
public List<ConnectorTableLayoutResult> getTableLayouts(
262+
public ConnectorTableLayoutResult getTableLayoutForConstraint(
263263
ConnectorSession session,
264264
ConnectorTableHandle table,
265265
Constraint<ColumnHandle> constraint,
266266
Optional<Set<ColumnHandle>> desiredColumns)
267267
{
268268
AccumuloTableHandle tableHandle = (AccumuloTableHandle) table;
269269
ConnectorTableLayout layout = new ConnectorTableLayout(new AccumuloTableLayoutHandle(tableHandle, constraint.getSummary()));
270-
return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary()));
270+
return new ConnectorTableLayoutResult(layout, constraint.getSummary());
271271
}
272272

273273
@Override

presto-atop/src/main/java/com/facebook/presto/atop/AtopMetadata.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -85,7 +85,8 @@ public ConnectorTableHandle getTableHandle(ConnectorSession session, SchemaTable
8585
}
8686

8787
@Override
88-
public List<ConnectorTableLayoutResult> getTableLayouts(ConnectorSession session,
88+
public ConnectorTableLayoutResult getTableLayoutForConstraint(
89+
ConnectorSession session,
8990
ConnectorTableHandle table,
9091
Constraint<ColumnHandle> constraint,
9192
Optional<Set<ColumnHandle>> desiredColumns)
@@ -104,7 +105,7 @@ public List<ConnectorTableLayoutResult> getTableLayouts(ConnectorSession session
104105
}
105106
AtopTableLayoutHandle layoutHandle = new AtopTableLayoutHandle(tableHandle, startTimeDomain, endTimeDomain);
106107
ConnectorTableLayout tableLayout = getTableLayout(session, layoutHandle);
107-
return ImmutableList.of(new ConnectorTableLayoutResult(tableLayout, constraint.getSummary()));
108+
return new ConnectorTableLayoutResult(tableLayout, constraint.getSummary());
108109
}
109110

110111
@Override

presto-base-arrow-flight/src/main/java/com/facebook/plugin/arrow/ArrowMetadata.java

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -112,7 +112,11 @@ public Map<String, ColumnHandle> getColumnHandles(ConnectorSession session, Conn
112112
}
113113

114114
@Override
115-
public List<ConnectorTableLayoutResult> getTableLayouts(ConnectorSession session, ConnectorTableHandle table, Constraint<ColumnHandle> constraint, Optional<Set<ColumnHandle>> desiredColumns)
115+
public ConnectorTableLayoutResult getTableLayoutForConstraint(
116+
ConnectorSession session,
117+
ConnectorTableHandle table,
118+
Constraint<ColumnHandle> constraint,
119+
Optional<Set<ColumnHandle>> desiredColumns)
116120
{
117121
checkArgument(table instanceof ArrowTableHandle,
118122
"Invalid table handle: Expected an instance of ArrowTableHandle but received %s",
@@ -129,7 +133,7 @@ public List<ConnectorTableLayoutResult> getTableLayouts(ConnectorSession session
129133
}
130134

131135
ConnectorTableLayout layout = new ConnectorTableLayout(new ArrowTableLayoutHandle(tableHandle, columns, constraint.getSummary()));
132-
return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary()));
136+
return new ConnectorTableLayoutResult(layout, constraint.getSummary());
133137
}
134138

135139
@Override

presto-base-jdbc/src/main/java/com/facebook/presto/plugin/jdbc/JdbcMetadata.java

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -85,11 +85,15 @@ public JdbcTableHandle getTableHandle(ConnectorSession session, SchemaTableName
8585
}
8686

8787
@Override
88-
public List<ConnectorTableLayoutResult> getTableLayouts(ConnectorSession session, ConnectorTableHandle table, Constraint<ColumnHandle> constraint, Optional<Set<ColumnHandle>> desiredColumns)
88+
public ConnectorTableLayoutResult getTableLayoutForConstraint(
89+
ConnectorSession session,
90+
ConnectorTableHandle table,
91+
Constraint<ColumnHandle> constraint,
92+
Optional<Set<ColumnHandle>> desiredColumns)
8993
{
9094
JdbcTableHandle tableHandle = (JdbcTableHandle) table;
9195
ConnectorTableLayout layout = new ConnectorTableLayout(new JdbcTableLayoutHandle(session.getSqlFunctionProperties(), tableHandle, constraint.getSummary(), Optional.empty()));
92-
return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary()));
96+
return new ConnectorTableLayoutResult(layout, constraint.getSummary());
9397
}
9498

9599
@Override

presto-bigquery/src/main/java/com/facebook/presto/plugin/bigquery/BigQueryMetadata.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -127,7 +127,7 @@ public ConnectorTableHandle getTableHandle(ConnectorSession session, SchemaTable
127127
}
128128

129129
@Override
130-
public List<ConnectorTableLayoutResult> getTableLayouts(
130+
public ConnectorTableLayoutResult getTableLayoutForConstraint(
131131
ConnectorSession session,
132132
ConnectorTableHandle table,
133133
Constraint<ColumnHandle> constraint,
@@ -139,7 +139,7 @@ public List<ConnectorTableLayoutResult> getTableLayouts(
139139
bigQueryTableHandle = bigQueryTableHandle.withProjectedColumns(ImmutableList.copyOf(desiredColumns.get()));
140140
}
141141
BigQueryTableLayoutHandle bigQueryTableLayoutHandle = new BigQueryTableLayoutHandle(bigQueryTableHandle);
142-
return ImmutableList.of(new ConnectorTableLayoutResult(new ConnectorTableLayout(bigQueryTableLayoutHandle), constraint.getSummary()));
142+
return new ConnectorTableLayoutResult(new ConnectorTableLayout(bigQueryTableLayoutHandle), constraint.getSummary());
143143
}
144144

145145
@Override

presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleMetadata.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -253,7 +253,7 @@ public Optional<ConnectorOutputMetadata> finishInsert(ConnectorSession session,
253253
}
254254

255255
@Override
256-
public List<ConnectorTableLayoutResult> getTableLayouts(
256+
public ConnectorTableLayoutResult getTableLayoutForConstraint(
257257
ConnectorSession session,
258258
ConnectorTableHandle handle,
259259
Constraint<ColumnHandle> constraint,
@@ -266,7 +266,7 @@ public List<ConnectorTableLayoutResult> getTableLayouts(
266266
blackHoleHandle.getRowsPerPage(),
267267
blackHoleHandle.getFieldsLength(),
268268
blackHoleHandle.getPageProcessingDelay());
269-
return ImmutableList.of(new ConnectorTableLayoutResult(getTableLayout(session, layoutHandle), constraint.getSummary()));
269+
return new ConnectorTableLayoutResult(getTableLayout(session, layoutHandle), constraint.getSummary());
270270
}
271271

272272
@Override

presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraMetadata.java

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -201,7 +201,11 @@ public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTable
201201
}
202202

203203
@Override
204-
public List<ConnectorTableLayoutResult> getTableLayouts(ConnectorSession session, ConnectorTableHandle table, Constraint<ColumnHandle> constraint, Optional<Set<ColumnHandle>> desiredColumns)
204+
public ConnectorTableLayoutResult getTableLayoutForConstraint(
205+
ConnectorSession session,
206+
ConnectorTableHandle table,
207+
Constraint<ColumnHandle> constraint,
208+
Optional<Set<ColumnHandle>> desiredColumns)
205209
{
206210
CassandraTableHandle handle = (CassandraTableHandle) table;
207211
CassandraPartitionResult partitionResult = partitionManager.getPartitions(handle, constraint.getSummary());
@@ -224,7 +228,7 @@ public List<ConnectorTableLayoutResult> getTableLayouts(ConnectorSession session
224228
handle,
225229
partitionResult.getPartitions(),
226230
clusteringKeyPredicates));
227-
return ImmutableList.of(new ConnectorTableLayoutResult(layout, unenforcedConstraint));
231+
return new ConnectorTableLayoutResult(layout, unenforcedConstraint);
228232
}
229233

230234
@Override

presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraConnector.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -67,7 +67,6 @@
6767
import static com.facebook.presto.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING;
6868
import static com.facebook.presto.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED;
6969
import static com.google.common.base.Preconditions.checkArgument;
70-
import static com.google.common.collect.Iterables.getOnlyElement;
7170
import static java.util.Locale.ENGLISH;
7271
import static org.testng.Assert.assertEquals;
7372
import static org.testng.Assert.assertNull;
@@ -185,8 +184,8 @@ public void testGetRecords()
185184

186185
ConnectorTransactionHandle transaction = CassandraTransactionHandle.INSTANCE;
187186

188-
List<ConnectorTableLayoutResult> layouts = metadata.getTableLayouts(SESSION, tableHandle, Constraint.alwaysTrue(), Optional.empty());
189-
ConnectorTableLayoutHandle layout = getOnlyElement(layouts).getTableLayout().getHandle();
187+
ConnectorTableLayoutResult layoutResult = metadata.getTableLayoutForConstraint(SESSION, tableHandle, Constraint.alwaysTrue(), Optional.empty());
188+
ConnectorTableLayoutHandle layout = layoutResult.getTableLayout().getHandle();
190189
List<ConnectorSplit> splits = getAllSplits(splitManager.getSplits(transaction, SESSION, layout, new SplitSchedulingContext(UNGROUPED_SCHEDULING, false, WarningCollector.NOOP)));
191190

192191
long rowNumber = 0;

presto-clickhouse/src/main/java/com/facebook/presto/plugin/clickhouse/ClickHouseMetadata.java

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -84,11 +84,15 @@ public ClickHouseTableHandle getTableHandle(ConnectorSession session, SchemaTabl
8484
}
8585

8686
@Override
87-
public List<ConnectorTableLayoutResult> getTableLayouts(ConnectorSession session, ConnectorTableHandle table, Constraint<ColumnHandle> constraint, Optional<Set<ColumnHandle>> desiredColumns)
87+
public ConnectorTableLayoutResult getTableLayoutForConstraint(
88+
ConnectorSession session,
89+
ConnectorTableHandle table,
90+
Constraint<ColumnHandle> constraint,
91+
Optional<Set<ColumnHandle>> desiredColumns)
8892
{
8993
ClickHouseTableHandle tableHandle = (ClickHouseTableHandle) table;
9094
ConnectorTableLayout layout = new ConnectorTableLayout(new ClickHouseTableLayoutHandle(tableHandle, constraint.getSummary(), Optional.empty(), Optional.empty(), Optional.empty()));
91-
return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary()));
95+
return new ConnectorTableLayoutResult(layout, constraint.getSummary());
9296
}
9397

9498
@Override

presto-delta/src/main/java/com/facebook/presto/delta/DeltaMetadata.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -231,7 +231,7 @@ public DeltaTableHandle getTableHandle(ConnectorSession session, SchemaTableName
231231
}
232232

233233
@Override
234-
public List<ConnectorTableLayoutResult> getTableLayouts(
234+
public ConnectorTableLayoutResult getTableLayoutForConstraint(
235235
ConnectorSession session,
236236
ConnectorTableHandle table,
237237
Constraint<ColumnHandle> constraint,
@@ -259,7 +259,7 @@ public List<ConnectorTableLayoutResult> getTableLayouts(
259259
ImmutableList.of(),
260260
Optional.empty());
261261

262-
return ImmutableList.of(new ConnectorTableLayoutResult(newLayout, unenforcedPredicate));
262+
return new ConnectorTableLayoutResult(newLayout, unenforcedPredicate);
263263
}
264264

265265
@Override

0 commit comments

Comments
 (0)