Skip to content

Commit

Permalink
Rename hive.dynamic-filtering-probe-blocking-timeout config
Browse files Browse the repository at this point in the history
Changed name to hive.dynamic-filtering.wait-timeout for consistency
with other connectors and simplification of name.
  • Loading branch information
raunaqmorarka authored and findepi committed Nov 2, 2021
1 parent 9f202ea commit deaff22
Show file tree
Hide file tree
Showing 7 changed files with 26 additions and 25 deletions.
4 changes: 2 additions & 2 deletions docs/src/main/sphinx/connector/hive.rst
Original file line number Diff line number Diff line change
Expand Up @@ -828,8 +828,8 @@ in query and CPU time, if dynamic filtering is able to reduce the amount of scan

For the Hive connector, a table scan can be delayed for a configured amount of
time until the collection of dynamic filters by using the configuration property
``hive.dynamic-filtering-probe-blocking-timeout`` in the catalog file or the catalog
session property ``<hive-catalog>.dynamic_filtering_probe_blocking_timeout``.
``hive.dynamic-filtering.wait-timeout`` in the catalog file or the catalog
session property ``<hive-catalog>.dynamic_filtering_wait_timeout``.

Schema evolution
----------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -148,7 +148,7 @@ public class BackgroundHiveSplitLoader
private final AcidTransaction transaction;
private final TupleDomain<? extends ColumnHandle> compactEffectivePredicate;
private final DynamicFilter dynamicFilter;
private final long dynamicFilteringProbeBlockingTimeoutMillis;
private final long dynamicFilteringWaitTimeoutMillis;
private final TypeManager typeManager;
private final Optional<BucketSplitInfo> tableBucketInfo;
private final HdfsEnvironment hdfsEnvironment;
Expand Down Expand Up @@ -193,7 +193,7 @@ public BackgroundHiveSplitLoader(
Iterable<HivePartitionMetadata> partitions,
TupleDomain<? extends ColumnHandle> compactEffectivePredicate,
DynamicFilter dynamicFilter,
Duration dynamicFilteringProbeBlockingTimeout,
Duration dynamicFilteringWaitTimeout,
TypeManager typeManager,
Optional<BucketSplitInfo> tableBucketInfo,
ConnectorSession session,
Expand All @@ -212,7 +212,7 @@ public BackgroundHiveSplitLoader(
this.transaction = requireNonNull(transaction, "transaction is null");
this.compactEffectivePredicate = compactEffectivePredicate;
this.dynamicFilter = dynamicFilter;
this.dynamicFilteringProbeBlockingTimeoutMillis = dynamicFilteringProbeBlockingTimeout.toMillis();
this.dynamicFilteringWaitTimeoutMillis = dynamicFilteringWaitTimeout.toMillis();
this.typeManager = typeManager;
this.tableBucketInfo = tableBucketInfo;
this.loaderConcurrency = loaderConcurrency;
Expand Down Expand Up @@ -262,7 +262,7 @@ public TaskStatus process()
// Block until one of below conditions is met:
// 1. Completion of DynamicFilter
// 2. Timeout after waiting for the configured time
long timeLeft = dynamicFilteringProbeBlockingTimeoutMillis - stopwatch.elapsed(MILLISECONDS);
long timeLeft = dynamicFilteringWaitTimeoutMillis - stopwatch.elapsed(MILLISECONDS);
if (timeLeft > 0 && dynamicFilter.isAwaitable()) {
future = asVoid(toListenableFuture(dynamicFilter.isBlocked().orTimeout(timeLeft, MILLISECONDS)));
return TaskStatus.continueOn(future);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,7 @@ public class HiveConfig

private boolean projectionPushdownEnabled = true;

private Duration dynamicFilteringProbeBlockingTimeout = new Duration(0, MINUTES);
private Duration dynamicFilteringWaitTimeout = new Duration(0, MINUTES);

private HiveTimestampPrecision timestampPrecision = HiveTimestampPrecision.DEFAULT_PRECISION;

Expand Down Expand Up @@ -1039,16 +1039,17 @@ public HiveConfig setProjectionPushdownEnabled(boolean projectionPushdownEnabled
}

@NotNull
public Duration getDynamicFilteringProbeBlockingTimeout()
public Duration getDynamicFilteringWaitTimeout()
{
return dynamicFilteringProbeBlockingTimeout;
return dynamicFilteringWaitTimeout;
}

@Config("hive.dynamic-filtering-probe-blocking-timeout")
@ConfigDescription("Duration to wait for completion of dynamic filters during split generation for probe side table")
public HiveConfig setDynamicFilteringProbeBlockingTimeout(Duration dynamicFilteringProbeBlockingTimeout)
@Config("hive.dynamic-filtering.wait-timeout")
@LegacyConfig("hive.dynamic-filtering-probe-blocking-timeout")
@ConfigDescription("Duration to wait for completion of dynamic filters during split generation")
public HiveConfig setDynamicFilteringWaitTimeout(Duration dynamicFilteringWaitTimeout)
{
this.dynamicFilteringProbeBlockingTimeout = dynamicFilteringProbeBlockingTimeout;
this.dynamicFilteringWaitTimeout = dynamicFilteringWaitTimeout;
return this;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ public final class HiveSessionProperties
private static final String PROJECTION_PUSHDOWN_ENABLED = "projection_pushdown_enabled";
private static final String TIMESTAMP_PRECISION = "timestamp_precision";
private static final String PARQUET_OPTIMIZED_WRITER_ENABLED = "experimental_parquet_optimized_writer_enabled";
private static final String DYNAMIC_FILTERING_PROBE_BLOCKING_TIMEOUT = "dynamic_filtering_probe_blocking_timeout";
private static final String DYNAMIC_FILTERING_WAIT_TIMEOUT = "dynamic_filtering_wait_timeout";
private static final String OPTIMIZE_SYMLINK_LISTING = "optimize_symlink_listing";
private static final String LEGACY_HIVE_VIEW_TRANSLATION = "legacy_hive_view_translation";
public static final String SIZE_BASED_SPLIT_WEIGHTS_ENABLED = "size_based_split_weights_enabled";
Expand Down Expand Up @@ -449,9 +449,9 @@ public HiveSessionProperties(
parquetWriterConfig.isParquetOptimizedWriterEnabled(),
false),
durationProperty(
DYNAMIC_FILTERING_PROBE_BLOCKING_TIMEOUT,
"Duration to wait for completion of dynamic filters during split generation for probe side table",
hiveConfig.getDynamicFilteringProbeBlockingTimeout(),
DYNAMIC_FILTERING_WAIT_TIMEOUT,
"Duration to wait for completion of dynamic filters during split generation",
hiveConfig.getDynamicFilteringWaitTimeout(),
false),
booleanProperty(
OPTIMIZE_SYMLINK_LISTING,
Expand Down Expand Up @@ -775,9 +775,9 @@ public static boolean isParquetOptimizedWriterEnabled(ConnectorSession session)
return session.getProperty(PARQUET_OPTIMIZED_WRITER_ENABLED, Boolean.class);
}

public static Duration getDynamicFilteringProbeBlockingTimeout(ConnectorSession session)
public static Duration getDynamicFilteringWaitTimeout(ConnectorSession session)
{
return session.getProperty(DYNAMIC_FILTERING_PROBE_BLOCKING_TIMEOUT, Duration.class);
return session.getProperty(DYNAMIC_FILTERING_WAIT_TIMEOUT, Duration.class);
}

public static boolean isOptimizeSymlinkListing(ConnectorSession session)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@
import static io.trino.plugin.hive.HiveErrorCode.HIVE_PARTITION_DROPPED_DURING_QUERY;
import static io.trino.plugin.hive.HiveErrorCode.HIVE_PARTITION_SCHEMA_MISMATCH;
import static io.trino.plugin.hive.HivePartition.UNPARTITIONED_ID;
import static io.trino.plugin.hive.HiveSessionProperties.getDynamicFilteringProbeBlockingTimeout;
import static io.trino.plugin.hive.HiveSessionProperties.getDynamicFilteringWaitTimeout;
import static io.trino.plugin.hive.HiveSessionProperties.isIgnoreAbsentPartitions;
import static io.trino.plugin.hive.HiveSessionProperties.isOptimizeSymlinkListing;
import static io.trino.plugin.hive.HiveSessionProperties.isPropagateTableScanSortingProperties;
Expand Down Expand Up @@ -235,7 +235,7 @@ public ConnectorSplitSource getSplits(
hivePartitions,
hiveTable.getCompactEffectivePredicate(),
dynamicFilter,
getDynamicFilteringProbeBlockingTimeout(session),
getDynamicFilteringWaitTimeout(session),
typeManager,
createBucketSplitInfo(bucketHandle, bucketFilter),
session,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ public void testDefaults()
.setQueryPartitionFilterRequired(false)
.setQueryPartitionFilterRequiredSchemas("")
.setProjectionPushdownEnabled(true)
.setDynamicFilteringProbeBlockingTimeout(new Duration(0, TimeUnit.MINUTES))
.setDynamicFilteringWaitTimeout(new Duration(0, TimeUnit.MINUTES))
.setTimestampPrecision(HiveTimestampPrecision.DEFAULT_PRECISION)
.setOptimizeSymlinkListing(true)
.setLegacyHiveViewTranslation(false)
Expand Down Expand Up @@ -178,7 +178,7 @@ public void testExplicitPropertyMappings()
.put("hive.query-partition-filter-required", "true")
.put("hive.query-partition-filter-required-schemas", "foo, bar")
.put("hive.projection-pushdown-enabled", "false")
.put("hive.dynamic-filtering-probe-blocking-timeout", "10s")
.put("hive.dynamic-filtering.wait-timeout", "10s")
.put("hive.timestamp-precision", "NANOSECONDS")
.put("hive.optimize-symlink-listing", "false")
.put("hive.legacy-hive-view-translation", "true")
Expand Down Expand Up @@ -252,7 +252,7 @@ public void testExplicitPropertyMappings()
.setQueryPartitionFilterRequired(true)
.setQueryPartitionFilterRequiredSchemas("foo, bar")
.setProjectionPushdownEnabled(false)
.setDynamicFilteringProbeBlockingTimeout(new Duration(10, TimeUnit.SECONDS))
.setDynamicFilteringWaitTimeout(new Duration(10, TimeUnit.SECONDS))
.setTimestampPrecision(HiveTimestampPrecision.NANOSECONDS)
.setOptimizeSymlinkListing(false)
.setLegacyHiveViewTranslation(true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ protected QueryRunner createQueryRunner()
{
return HiveQueryRunner.builder()
.setExtraProperties(EXTRA_PROPERTIES)
.setHiveProperties(ImmutableMap.of("hive.dynamic-filtering-probe-blocking-timeout", "1h"))
.setHiveProperties(ImmutableMap.of("hive.dynamic-filtering.wait-timeout", "1h"))
.setInitialTables(REQUIRED_TABLES)
.build();
}
Expand Down

0 comments on commit deaff22

Please sign in to comment.