Skip to content

Commit

Permalink
Remove ConnectorTableLayoutHandle from TableHandle
Browse files Browse the repository at this point in the history
  • Loading branch information
Praveen2112 committed Jan 13, 2022
1 parent 31476e3 commit eb32a25
Show file tree
Hide file tree
Showing 34 changed files with 64 additions and 146 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -327,16 +327,14 @@ public Optional<TableHandle> getTableHandle(Session session, QualifiedObjectName
.map(connectorTableHandle -> new TableHandle(
catalogName,
connectorTableHandle,
catalogMetadata.getTransactionHandleFor(catalogName),
Optional.empty()));
catalogMetadata.getTransactionHandleFor(catalogName)));
}

return Optional.ofNullable(metadata.getTableHandle(connectorSession, table.asSchemaTableName()))
.map(connectorTableHandle -> new TableHandle(
catalogName,
connectorTableHandle,
catalogMetadata.getTransactionHandleFor(catalogName),
Optional.empty()));
catalogMetadata.getTransactionHandleFor(catalogName)));
});
}

Expand All @@ -356,8 +354,7 @@ public Optional<TableHandle> getTableHandleForStatisticsCollection(Session sessi
return Optional.of(new TableHandle(
catalogName,
tableHandle,
catalogMetadata.getTransactionHandleFor(catalogName),
Optional.empty()));
catalogMetadata.getTransactionHandleFor(catalogName)));
}
}
return Optional.empty();
Expand Down Expand Up @@ -460,12 +457,11 @@ public TableHandle makeCompatiblePartitioning(Session session, TableHandle table
ConnectorMetadata metadata = catalogMetadata.getMetadataFor(catalogName);
ConnectorTransactionHandle transaction = catalogMetadata.getTransactionHandleFor(catalogName);

verify(tableHandle.getLayout().isEmpty(), "layout should not be present");
ConnectorTableHandle newTableHandle = metadata.makeCompatiblePartitioning(
session.toConnectorSession(catalogName),
tableHandle.getConnectorHandle(),
partitioningHandle.getConnectorHandle());
return new TableHandle(catalogName, newTableHandle, transaction, Optional.empty());
return new TableHandle(catalogName, newTableHandle, transaction);
}

@Override
Expand Down Expand Up @@ -1029,7 +1025,7 @@ public Optional<TableHandle> applyDelete(Session session, TableHandle table)

ConnectorSession connectorSession = session.toConnectorSession(catalogName);
return metadata.applyDelete(connectorSession, table.getConnectorHandle())
.map(newHandle -> new TableHandle(catalogName, newHandle, table.getTransaction(), Optional.empty()));
.map(newHandle -> new TableHandle(catalogName, newHandle, table.getTransaction()));
}

@Override
Expand All @@ -1039,8 +1035,6 @@ public OptionalLong executeDelete(Session session, TableHandle table)
ConnectorMetadata metadata = getMetadataForWrite(session, catalogName);
ConnectorSession connectorSession = session.toConnectorSession(catalogName);

checkArgument(table.getLayout().isEmpty(), "table layout should not be present");

return metadata.executeDelete(connectorSession, table.getConnectorHandle());
}

Expand All @@ -1050,7 +1044,7 @@ public TableHandle beginDelete(Session session, TableHandle tableHandle)
CatalogName catalogName = tableHandle.getCatalogName();
ConnectorMetadata metadata = getMetadataForWrite(session, catalogName);
ConnectorTableHandle newHandle = metadata.beginDelete(session.toConnectorSession(catalogName), tableHandle.getConnectorHandle(), getRetryPolicy(session).getRetryMode());
return new TableHandle(tableHandle.getCatalogName(), newHandle, tableHandle.getTransaction(), tableHandle.getLayout());
return new TableHandle(tableHandle.getCatalogName(), newHandle, tableHandle.getTransaction());
}

@Override
Expand All @@ -1067,7 +1061,7 @@ public TableHandle beginUpdate(Session session, TableHandle tableHandle, List<Co
CatalogName catalogName = tableHandle.getCatalogName();
ConnectorMetadata metadata = getMetadataForWrite(session, catalogName);
ConnectorTableHandle newHandle = metadata.beginUpdate(session.toConnectorSession(catalogName), tableHandle.getConnectorHandle(), updatedColumns, getRetryPolicy(session).getRetryMode());
return new TableHandle(tableHandle.getCatalogName(), newHandle, tableHandle.getTransaction(), tableHandle.getLayout());
return new TableHandle(tableHandle.getCatalogName(), newHandle, tableHandle.getTransaction());
}

@Override
Expand Down Expand Up @@ -1565,7 +1559,7 @@ public Optional<LimitApplicationResult<TableHandle>> applyLimit(Session session,
ConnectorSession connectorSession = session.toConnectorSession(catalogName);
return metadata.applyLimit(connectorSession, table.getConnectorHandle(), limit)
.map(result -> new LimitApplicationResult<>(
new TableHandle(catalogName, result.getHandle(), table.getTransaction(), Optional.empty()),
new TableHandle(catalogName, result.getHandle(), table.getTransaction()),
result.isLimitGuaranteed(),
result.isPrecalculateStatistics()));
}
Expand All @@ -1585,8 +1579,7 @@ public Optional<SampleApplicationResult<TableHandle>> applySample(Session sessio
.map(result -> new SampleApplicationResult<>(new TableHandle(
catalogName,
result.getHandle(),
table.getTransaction(),
Optional.empty()),
table.getTransaction()),
result.isPrecalculateStatistics()));
}

Expand Down Expand Up @@ -1614,7 +1607,7 @@ public Optional<AggregationApplicationResult<TableHandle>> applyAggregation(
verifyProjection(table, result.getProjections(), result.getAssignments(), aggregations.size());

return new AggregationApplicationResult<>(
new TableHandle(catalogName, result.getHandle(), table.getTransaction(), Optional.empty()),
new TableHandle(catalogName, result.getHandle(), table.getTransaction()),
result.getProjections(),
result.getAssignments(),
result.getGroupingColumnMapping(),
Expand Down Expand Up @@ -1676,8 +1669,7 @@ public Optional<JoinApplicationResult<TableHandle>> applyJoin(
new TableHandle(
catalogName,
result.getTableHandle(),
transaction,
Optional.empty()),
transaction),
result.getLeftColumnHandles(),
result.getRightColumnHandles(),
result.isPrecalculateStatistics());
Expand All @@ -1702,7 +1694,7 @@ public Optional<TopNApplicationResult<TableHandle>> applyTopN(
ConnectorSession connectorSession = session.toConnectorSession(catalogName);
return metadata.applyTopN(connectorSession, table.getConnectorHandle(), topNCount, sortItems, assignments)
.map(result -> new TopNApplicationResult<>(
new TableHandle(catalogName, result.getHandle(), table.getTransaction(), Optional.empty()),
new TableHandle(catalogName, result.getHandle(), table.getTransaction()),
result.isTopNGuaranteed(),
result.isPrecalculateStatistics()));
}
Expand Down Expand Up @@ -1751,7 +1743,7 @@ public Optional<ConstraintApplicationResult<TableHandle>> applyFilter(Session se
ConnectorSession connectorSession = session.toConnectorSession(catalogName);
return metadata.applyFilter(connectorSession, table.getConnectorHandle(), constraint)
.map(result -> new ConstraintApplicationResult<>(
new TableHandle(catalogName, result.getHandle(), table.getTransaction(), Optional.empty()),
new TableHandle(catalogName, result.getHandle(), table.getTransaction()),
result.getRemainingFilter(),
result.isPrecalculateStatistics()));
}
Expand All @@ -1772,7 +1764,7 @@ public Optional<ProjectionApplicationResult<TableHandle>> applyProjection(Sessio
verifyProjection(table, result.getProjections(), result.getAssignments(), projections.size());

return new ProjectionApplicationResult<>(
new TableHandle(catalogName, result.getHandle(), table.getTransaction(), Optional.empty()),
new TableHandle(catalogName, result.getHandle(), table.getTransaction()),
result.getProjections(),
result.getAssignments(),
result.isPrecalculateStatistics());
Expand Down
24 changes: 4 additions & 20 deletions core/trino-main/src/main/java/io/trino/metadata/TableHandle.java
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,9 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import io.trino.connector.CatalogName;
import io.trino.spi.connector.ConnectorTableHandle;
import io.trino.spi.connector.ConnectorTableLayoutHandle;
import io.trino.spi.connector.ConnectorTransactionHandle;

import java.util.Objects;
import java.util.Optional;

import static java.util.Objects.requireNonNull;

Expand All @@ -31,21 +29,15 @@ public final class TableHandle
private final ConnectorTableHandle connectorHandle;
private final ConnectorTransactionHandle transaction;

// Table layouts are deprecated, but we keep this here to hide the notion of layouts
// from the engine. TODO: it should be removed once table layouts are finally deleted
private final Optional<ConnectorTableLayoutHandle> layout;

@JsonCreator
public TableHandle(
@JsonProperty("catalogName") CatalogName catalogName,
@JsonProperty("connectorHandle") ConnectorTableHandle connectorHandle,
@JsonProperty("transaction") ConnectorTransactionHandle transaction,
@JsonProperty("layout") Optional<ConnectorTableLayoutHandle> layout)
@JsonProperty("transaction") ConnectorTransactionHandle transaction)
{
this.catalogName = requireNonNull(catalogName, "catalogName is null");
this.connectorHandle = requireNonNull(connectorHandle, "connectorHandle is null");
this.transaction = requireNonNull(transaction, "transaction is null");
this.layout = requireNonNull(layout, "layout is null");
}

@JsonProperty
Expand All @@ -60,12 +52,6 @@ public ConnectorTableHandle getConnectorHandle()
return connectorHandle;
}

@JsonProperty
public Optional<ConnectorTableLayoutHandle> getLayout()
{
return layout;
}

@JsonProperty
public ConnectorTransactionHandle getTransaction()
{
Expand All @@ -77,8 +63,7 @@ public TableHandle withConnectorHandle(ConnectorTableHandle connectorHandle)
return new TableHandle(
catalogName,
connectorHandle,
transaction,
layout);
transaction);
}

@Override
Expand All @@ -99,13 +84,12 @@ public boolean equals(Object o)
TableHandle other = (TableHandle) o;
return Objects.equals(catalogName, other.catalogName) &&
Objects.equals(connectorHandle, other.connectorHandle) &&
Objects.equals(transaction, other.transaction) &&
Objects.equals(layout, other.layout);
Objects.equals(transaction, other.transaction);
}

@Override
public int hashCode()
{
return Objects.hash(catalogName, connectorHandle, transaction, layout);
return Objects.hash(catalogName, connectorHandle, transaction);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,12 @@
import io.trino.metadata.TableHandle;
import io.trino.testing.TestingMetadata.TestingTableHandle;

import java.util.Optional;

public final class TestingHandles
{
private TestingHandles() {}

public static final TableHandle TEST_TABLE_HANDLE = new TableHandle(
new CatalogName("test"),
new TestingTableHandle(),
TestingTransactionHandle.create(),
Optional.of(TestingHandle.INSTANCE));
TestingTransactionHandle.create());
}
Original file line number Diff line number Diff line change
Expand Up @@ -793,7 +793,7 @@ private TableScanNode tableScan(String id, String... symbols)
TpchTableHandle tableHandle = new TpchTableHandle("sf1", "orders", 1.0);
return new TableScanNode(
new PlanNodeId(id),
new TableHandle(new CatalogName("tpch"), tableHandle, INSTANCE, Optional.empty()),
new TableHandle(new CatalogName("tpch"), tableHandle, INSTANCE),
symbolsList,
assignments.build(),
TupleDomain.all(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -233,8 +233,7 @@ public Optional<TableHandle> getTableHandle(Session session, QualifiedObjectName
.map(tableMetadata -> new TableHandle(
new CatalogName(CATALOG_NAME),
new TestingTableHandle(tableName.asSchemaTableName()),
TestingConnectorTransactionHandle.INSTANCE,
Optional.empty()));
TestingConnectorTransactionHandle.INSTANCE));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -276,8 +276,7 @@ public Optional<TableHandle> getTableHandle(Session session, QualifiedObjectName
new TableHandle(
new CatalogName(CATALOG_NAME),
new TestingTableHandle(tableName.asSchemaTableName()),
TestingConnectorTransactionHandle.INSTANCE,
Optional.empty()));
TestingConnectorTransactionHandle.INSTANCE));
}
return Optional.empty();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -333,8 +333,7 @@ public Optional<TableHandle> getTableHandle(Session session, QualifiedObjectName
new TableHandle(
new CatalogName(CATALOG_NAME),
new TestingTableHandle(tableName.asSchemaTableName()),
TestingConnectorTransactionHandle.INSTANCE,
Optional.empty()));
TestingConnectorTransactionHandle.INSTANCE));
}
return Optional.empty();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1246,8 +1246,7 @@ private static TableHandle makeTableHandle(TupleDomain<ColumnHandle> predicate)
return new TableHandle(
new CatalogName("test"),
new PredicatedTableHandle(predicate),
TestingTransactionHandle.create(),
Optional.empty());
TestingTransactionHandle.create());
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -274,7 +274,6 @@ private static TableHandle tableHandle(ConnectorTableHandle connectorTableHandle
return new TableHandle(
new CatalogName(MOCK_CATALOG),
connectorTableHandle,
TestingTransactionHandle.create(),
Optional.empty());
TestingTransactionHandle.create());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -84,8 +84,7 @@ private static TableHandle createTableHandle(ConnectorTableHandle tableHandle)
return new TableHandle(
new CatalogName(MOCK_CATALOG),
tableHandle,
TestingTransactionHandle.create(),
Optional.empty());
TestingTransactionHandle.create());
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,6 @@
import io.trino.sql.tree.SymbolReference;
import org.testng.annotations.Test;

import java.util.Optional;

import static io.trino.plugin.tpch.TpchMetadata.TINY_SCALE_FACTOR;
import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME;
import static io.trino.spi.type.DoubleType.DOUBLE;
Expand Down Expand Up @@ -164,8 +162,7 @@ public void testDoesNotFireOnNestedNonCountAggregate()
new TableHandle(
new CatalogName("local"),
new TpchTableHandle(TINY_SCHEMA_NAME, "orders", TINY_SCALE_FACTOR),
TpchTransactionHandle.INSTANCE,
Optional.empty()),
TpchTransactionHandle.INSTANCE),
ImmutableList.of(totalPrice),
ImmutableMap.of(totalPrice, new TpchColumnHandle(totalPrice.getName(), DOUBLE))))));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
import io.trino.sql.planner.plan.PlanNode;
import org.testng.annotations.Test;

import java.util.Optional;
import java.util.function.Predicate;

import static com.google.common.collect.ImmutableList.toImmutableList;
Expand Down Expand Up @@ -84,8 +83,7 @@ private static PlanNode buildProjectedIndexSource(PlanBuilder p, Predicate<Symbo
new TableHandle(
new CatalogName("local"),
new TpchTableHandle(TINY_SCHEMA_NAME, "orders", TINY_SCALE_FACTOR),
TpchTransactionHandle.INSTANCE,
Optional.empty()),
TpchTransactionHandle.INSTANCE),
ImmutableSet.of(orderkey, custkey),
ImmutableList.of(orderkey, custkey, totalprice),
ImmutableMap.of(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,8 +76,7 @@ public void testNotAllOutputsReferenced()
new TableHandle(
new CatalogName("local"),
new TpchTableHandle(TINY_SCHEMA_NAME, "orders", TINY_SCALE_FACTOR),
TpchTransactionHandle.INSTANCE,
Optional.empty()),
TpchTransactionHandle.INSTANCE),
ImmutableList.of(orderdate, totalprice),
ImmutableMap.of(
orderdate, new TpchColumnHandle(orderdate.getName(), DATE),
Expand All @@ -104,8 +103,7 @@ public void testPruneEnforcedConstraint()
new TableHandle(
new CatalogName("local"),
new TpchTableHandle(TINY_SCHEMA_NAME, "orders", TINY_SCALE_FACTOR),
TpchTransactionHandle.INSTANCE,
Optional.empty()),
TpchTransactionHandle.INSTANCE),
List.of(orderdate, totalprice),
Map.of(
orderdate, orderdateHandle,
Expand Down Expand Up @@ -172,8 +170,7 @@ public void testPushColumnPruningProjection()
new TableHandle(
new CatalogName(mockCatalog),
new MockConnectorTableHandle(testSchemaTable),
MockConnectorTransactionHandle.INSTANCE,
Optional.empty()),
MockConnectorTransactionHandle.INSTANCE),
ImmutableList.of(symbolA, symbolB),
ImmutableMap.of(
symbolA, columnHandleA,
Expand Down
Loading

0 comments on commit eb32a25

Please sign in to comment.