Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Remove legacy non-gRPC client from Pinot #22213

Merged
merged 1 commit into from
May 31, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 1 addition & 3 deletions docs/src/main/sphinx/connector/pinot.md
Original file line number Diff line number Diff line change
Expand Up @@ -44,8 +44,7 @@ This can be the ip or the FDQN, the url scheme (`http://`) is optional.
| `pinot.broker.authentication.type` | No | Pinot authentication method for broker requests. Allowed values are `NONE` and `PASSWORD` - defaults to `NONE` which is no authentication. |
| `pinot.broker.authentication.user` | No | Broker username for basic authentication method. |
| `pinot.broker.authentication.password` | No | Broker password for basic authentication method. |
| `pinot.max-rows-per-split-for-segment-queries` | No | Fail query if Pinot server split returns more rows than configured, default to `50,000` for non-gRPC connection, `2,147,483,647` for gRPC connection. |
| `pinot.estimated-size-in-bytes-for-non-numeric-column` | No | Estimated byte size for non-numeric column for page pre-allocation in non-gRPC connection, default is `20`. |
| `pinot.max-rows-per-split-for-segment-queries` | No | Fail query if Pinot server split returns more rows than configured, default to `2,147,483,647`. |
| `pinot.prefer-broker-queries` | No | Pinot query plan prefers to query Pinot broker, default is `true`. |
| `pinot.forbid-segment-queries` | No | Forbid parallel querying and force all querying to happen via the broker, default is `false`. |
| `pinot.segments-per-split` | No | The number of segments processed in a split. Setting this higher reduces the number of requests made to Pinot. This is useful for smaller Pinot clusters, default is `1`. |
Expand All @@ -69,7 +68,6 @@ If `pinot.controller-urls` uses `https` scheme then TLS is enabled for all conne

| Property name | Required | Description |
| ------------------------------------- | -------- | -------------------------------------------------------------------- |
| `pinot.grpc.enabled` | No | Use gRPC endpoint for Pinot server queries, default is `true`. |
| `pinot.grpc.port` | No | Pinot gRPC port, default to `8090`. |
| `pinot.grpc.max-inbound-message-size` | No | Max inbound message bytes when init gRPC client, default is `128MB`. |
| `pinot.grpc.use-plain-text` | No | Use plain text for gRPC communication, default to `true`. |
Expand Down
5 changes: 0 additions & 5 deletions plugin/trino-pinot/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -122,11 +122,6 @@
<artifactId>jakarta.validation-api</artifactId>
</dependency>

<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>

<dependency>
<groupId>org.apache.helix</groupId>
<artifactId>helix-core</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,9 @@
"pinot.max-backlog-per-server",
"pinot.max-connections-per-server",
"pinot.min-connections-per-server",
"pinot.request-timeout"
"pinot.request-timeout",
"pinot.estimated-size-in-bytes-for-non-numeric-column",
"pinot.grpc.enabled",
})
public class PinotConfig
{
Expand All @@ -55,7 +57,6 @@ public class PinotConfig

private Duration connectionTimeout = new Duration(1, TimeUnit.MINUTES);

private int estimatedSizeInBytesForNonNumericColumn = 20;
private Duration metadataCacheExpiry = new Duration(2, TimeUnit.MINUTES);

private boolean preferBrokerQueries;
Expand All @@ -66,7 +67,6 @@ public class PinotConfig
private int maxRowsForBrokerQueries = 50_000;
private boolean aggregationPushdownEnabled = true;
private boolean countDistinctPushdownEnabled = true;
private boolean grpcEnabled = true;
private boolean proxyEnabled;
private DataSize targetSegmentPageSize = DataSize.of(1, MEGABYTE);

Expand Down Expand Up @@ -126,18 +126,6 @@ public PinotConfig setMetadataCacheExpiry(Duration metadataCacheExpiry)
return this;
}

public int getEstimatedSizeInBytesForNonNumericColumn()
{
return estimatedSizeInBytesForNonNumericColumn;
}

@Config("pinot.estimated-size-in-bytes-for-non-numeric-column")
public PinotConfig setEstimatedSizeInBytesForNonNumericColumn(int estimatedSizeInBytesForNonNumericColumn)
{
this.estimatedSizeInBytesForNonNumericColumn = estimatedSizeInBytesForNonNumericColumn;
return this;
}

public boolean isPreferBrokerQueries()
{
return preferBrokerQueries;
Expand Down Expand Up @@ -244,18 +232,6 @@ public PinotConfig setCountDistinctPushdownEnabled(boolean countDistinctPushdown
return this;
}

public boolean isGrpcEnabled()
{
return grpcEnabled;
}

@Config("pinot.grpc.enabled")
public PinotConfig setGrpcEnabled(boolean grpcEnabled)
{
this.grpcEnabled = grpcEnabled;
return this;
}

public boolean isTlsEnabled()
{
return "https".equalsIgnoreCase(getControllerUrls().get(0).getScheme());
Expand Down Expand Up @@ -301,13 +277,4 @@ public boolean allUrlSchemesEqual()
.distinct()
.count() == 1;
}

@AssertTrue(message = "Using the rest proxy requires GRPC to be enabled by setting pinot.grpc.enabled=true")
public boolean proxyRestAndGrpcAreRequired()
{
if (proxyEnabled) {
return grpcEnabled;
}
return true;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,6 @@
import io.trino.plugin.pinot.client.PinotGrpcServerQueryClientConfig;
import io.trino.plugin.pinot.client.PinotGrpcServerQueryClientTlsConfig;
import io.trino.plugin.pinot.client.PinotHostMapper;
import io.trino.plugin.pinot.client.PinotLegacyDataFetcher;
import io.trino.plugin.pinot.client.PinotLegacyServerQueryClientConfig;
import io.trino.spi.NodeManager;
import io.trino.spi.connector.ConnectorNodePartitioningProvider;
import org.apache.pinot.common.utils.DataSchema;
Expand Down Expand Up @@ -97,11 +95,7 @@ public void setup(Binder binder)
binder.bind(ConnectorNodePartitioningProvider.class).to(PinotNodePartitioningProvider.class).in(Scopes.SINGLETON);
newOptionalBinder(binder, PinotHostMapper.class).setDefault().to(IdentityPinotHostMapper.class).in(Scopes.SINGLETON);

install(conditionalModule(
PinotConfig.class,
PinotConfig::isGrpcEnabled,
new PinotGrpcModule(),
new LegacyClientModule()));
install(new PinotGrpcModule());
}

public static final class DataSchemaDeserializer
Expand Down Expand Up @@ -144,15 +138,4 @@ public void setup(Binder binder)
}));
}
}

public static class LegacyClientModule
extends AbstractConfigurationAwareModule
{
@Override
public void setup(Binder binder)
{
configBinder(binder).bindConfig(PinotLegacyServerQueryClientConfig.class);
binder.bind(PinotDataFetcher.Factory.class).to(PinotLegacyDataFetcher.Factory.class).in(Scopes.SINGLETON);
}
}
}
Loading
Loading