Skip to content

Commit

Permalink
Remove deprecated SPI elements
Browse files Browse the repository at this point in the history
They have been deprecated while removing grouped execution
  • Loading branch information
skrzypo987 authored and raunaqmorarka committed Jul 20, 2022
1 parent a7455f2 commit 44bb41a
Show file tree
Hide file tree
Showing 8 changed files with 1 addition and 138 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,20 +23,6 @@ public interface ConnectorNodePartitioningProvider
// TODO: Use ConnectorPartitionHandle (instead of int) to represent individual buckets.
// Currently, it's mixed. listPartitionHandles used CPartitionHandle whereas the other functions used int.

/**
* Returns a list of all partitions associated with the provided {@code partitioningHandle}.
* <p>
* This method must be implemented for connectors that support addressable split discovery.
* The partitions return here will be used as address for the purpose of split discovery.
*
* @deprecated The method is not used. Implementations can be simply removed
*/
@Deprecated
default List<ConnectorPartitionHandle> listPartitionHandles(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorPartitioningHandle partitioningHandle)
{
throw new UnsupportedOperationException();
}

ConnectorBucketNodeMap getBucketNodeMap(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorPartitioningHandle partitioningHandle);

ToIntFunction<ConnectorSplit> getSplitBucketFunction(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorPartitioningHandle partitioningHandle);
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -13,36 +13,15 @@
*/
package io.trino.spi.connector;

import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING;

public interface ConnectorSplitManager
{
@Deprecated
default ConnectorSplitSource getSplits(
ConnectorTransactionHandle transaction,
ConnectorSession session,
ConnectorTableHandle table,
SplitSchedulingStrategy splitSchedulingStrategy,
DynamicFilter dynamicFilter,
Constraint constraint)
{
throw new UnsupportedOperationException();
}

default ConnectorSplitSource getSplits(
ConnectorTransactionHandle transaction,
ConnectorSession session,
ConnectorTableHandle table,
DynamicFilter dynamicFilter,
Constraint constraint)
{
return getSplits(transaction, session, table, UNGROUPED_SCHEDULING, dynamicFilter, constraint);
}

@Deprecated
enum SplitSchedulingStrategy
{
UNGROUPED_SCHEDULING,
GROUPED_SCHEDULING,
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,21 +18,14 @@
import java.util.Optional;
import java.util.concurrent.CompletableFuture;

import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED;
import static java.util.Objects.requireNonNull;

public interface ConnectorSplitSource
extends Closeable
{
@Deprecated
default CompletableFuture<ConnectorSplitBatch> getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize)
{
throw new UnsupportedOperationException();
}

default CompletableFuture<ConnectorSplitBatch> getNextBatch(int maxSize)
{
return getNextBatch(NOT_PARTITIONED, maxSize);
throw new UnsupportedOperationException();
}

@Override
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -51,18 +51,4 @@ public ConnectorSplitSource getSplits(
return delegate.getSplits(transaction, session, table, dynamicFilter, constraint);
}
}

@Override
public ConnectorSplitSource getSplits(
ConnectorTransactionHandle transaction,
ConnectorSession session,
ConnectorTableHandle table,
SplitSchedulingStrategy splitSchedulingStrategy,
DynamicFilter dynamicFilter,
Constraint constraint)
{
try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) {
return delegate.getSplits(transaction, session, table, splitSchedulingStrategy, dynamicFilter, constraint);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
package io.trino.plugin.base.classloader;

import io.trino.spi.classloader.ThreadContextClassLoader;
import io.trino.spi.connector.ConnectorPartitionHandle;
import io.trino.spi.connector.ConnectorSplitSource;

import javax.inject.Inject;
Expand All @@ -38,15 +37,6 @@ public ClassLoaderSafeConnectorSplitSource(@ForClassLoaderSafe ConnectorSplitSou
this.classLoader = requireNonNull(classLoader, "classLoader is null");
}

@Deprecated
@Override
public CompletableFuture<ConnectorSplitBatch> getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize)
{
try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) {
return delegate.getNextBatch(partitionHandle, maxSize);
}
}

@Override
public CompletableFuture<ConnectorSplitBatch> getNextBatch(int maxSize)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@
import io.trino.spi.connector.BucketFunction;
import io.trino.spi.connector.ConnectorBucketNodeMap;
import io.trino.spi.connector.ConnectorNodePartitioningProvider;
import io.trino.spi.connector.ConnectorPartitionHandle;
import io.trino.spi.connector.ConnectorPartitioningHandle;
import io.trino.spi.connector.ConnectorSession;
import io.trino.spi.connector.ConnectorSplit;
Expand Down Expand Up @@ -57,15 +56,6 @@ public BucketFunction getBucketFunction(
}
}

@Deprecated
@Override
public List<ConnectorPartitionHandle> listPartitionHandles(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorPartitioningHandle partitioningHandle)
{
try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) {
return delegate.listPartitionHandles(transactionHandle, session, partitioningHandle);
}
}

@Override
public ConnectorBucketNodeMap getBucketNodeMap(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorPartitioningHandle partitioningHandle)
{
Expand Down

0 comments on commit 44bb41a

Please sign in to comment.