mirror of https://github.com/apache/druid.git
Join filter pre-analysis simplifications and sanity checks. (#10104)
* Join filter pre-analysis simplifications and sanity checks. - At pre-analysis time, only compute pre-analysis for the innermost root query, since this is the one that will run on the join that involves the base datasource. Previously, pre-analyses were computed for multiple levels of the query, some of which were unnecessary. - Remove JoinFilterPreAnalysisGroup and join query level gathering code, since they existed to support precomputation of multiple pre-analyses. - Embed JoinFilterPreAnalysisKey into JoinFilterPreAnalysis and use it to sanity check at processing time that the correct pre-analysis was done. Tangentially related changes: - Remove prioritizeAndLaneQuery functionality from LocalQuerySegmentWalker. The computed priority and lanes were not being used. - Add "getBaseQuery" method to DataSourceAnalysis to support identification of the proper subquery for filter pre-analysis. * Fix compilation errors. * Adjust tests.
This commit is contained in:
parent
7b4edc93fc
commit
5faa897a34
|
@ -49,7 +49,9 @@ import org.apache.druid.segment.join.JoinConditionAnalysis;
|
|||
import org.apache.druid.segment.join.JoinTestHelper;
|
||||
import org.apache.druid.segment.join.JoinType;
|
||||
import org.apache.druid.segment.join.JoinableClause;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterAnalyzer;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
import org.apache.druid.segment.join.lookup.LookupJoinable;
|
||||
import org.apache.druid.segment.join.table.IndexedTableJoinable;
|
||||
|
@ -140,20 +142,26 @@ public class JoinAndLookupBenchmark
|
|||
)
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup preAnalysisGroupLookupStringKey = new JoinFilterPreAnalysisGroup(
|
||||
new JoinFilterRewriteConfig(
|
||||
false,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
),
|
||||
true
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysis preAnalysisLookupStringKey =
|
||||
JoinFilterAnalyzer.computeJoinFilterPreAnalysis(
|
||||
new JoinFilterPreAnalysisKey(
|
||||
new JoinFilterRewriteConfig(
|
||||
false,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
),
|
||||
joinableClausesLookupStringKey,
|
||||
VirtualColumns.EMPTY,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
||||
hashJoinLookupStringKeySegment = new HashJoinSegment(
|
||||
ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment),
|
||||
joinableClausesLookupStringKey,
|
||||
preAnalysisGroupLookupStringKey
|
||||
preAnalysisLookupStringKey
|
||||
);
|
||||
|
||||
List<JoinableClause> joinableClausesLookupLongKey = ImmutableList.of(
|
||||
|
@ -169,19 +177,25 @@ public class JoinAndLookupBenchmark
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup preAnalysisGroupLookupLongKey = new JoinFilterPreAnalysisGroup(
|
||||
new JoinFilterRewriteConfig(
|
||||
false,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
),
|
||||
true
|
||||
);
|
||||
JoinFilterPreAnalysis preAnalysisLookupLongKey =
|
||||
JoinFilterAnalyzer.computeJoinFilterPreAnalysis(
|
||||
new JoinFilterPreAnalysisKey(
|
||||
new JoinFilterRewriteConfig(
|
||||
false,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
),
|
||||
joinableClausesLookupLongKey,
|
||||
VirtualColumns.EMPTY,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
||||
hashJoinLookupLongKeySegment = new HashJoinSegment(
|
||||
ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment),
|
||||
joinableClausesLookupLongKey,
|
||||
preAnalysisGroupLookupLongKey
|
||||
preAnalysisLookupLongKey
|
||||
);
|
||||
|
||||
List<JoinableClause> joinableClausesIndexedTableStringKey = ImmutableList.of(
|
||||
|
@ -197,22 +211,28 @@ public class JoinAndLookupBenchmark
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup preAnalysisGroupIndexedStringKey = new JoinFilterPreAnalysisGroup(
|
||||
new JoinFilterRewriteConfig(
|
||||
false,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
),
|
||||
true
|
||||
);
|
||||
JoinFilterPreAnalysis preAnalysisIndexedStringKey =
|
||||
JoinFilterAnalyzer.computeJoinFilterPreAnalysis(
|
||||
new JoinFilterPreAnalysisKey(
|
||||
new JoinFilterRewriteConfig(
|
||||
false,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
),
|
||||
joinableClausesLookupLongKey,
|
||||
VirtualColumns.EMPTY,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
||||
hashJoinIndexedTableStringKeySegment = new HashJoinSegment(
|
||||
ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment),
|
||||
joinableClausesIndexedTableStringKey,
|
||||
preAnalysisGroupIndexedStringKey
|
||||
preAnalysisIndexedStringKey
|
||||
);
|
||||
|
||||
List<JoinableClause> joinableClausesIndexedTableLonggKey = ImmutableList.of(
|
||||
List<JoinableClause> joinableClausesIndexedTableLongKey = ImmutableList.of(
|
||||
new JoinableClause(
|
||||
prefix,
|
||||
new IndexedTableJoinable(JoinTestHelper.createCountriesIndexedTable()),
|
||||
|
@ -224,19 +244,26 @@ public class JoinAndLookupBenchmark
|
|||
)
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup preAnalysisGroupIndexedLongKey = new JoinFilterPreAnalysisGroup(
|
||||
new JoinFilterRewriteConfig(
|
||||
false,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
),
|
||||
true
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysis preAnalysisIndexedLongKey =
|
||||
JoinFilterAnalyzer.computeJoinFilterPreAnalysis(
|
||||
new JoinFilterPreAnalysisKey(
|
||||
new JoinFilterRewriteConfig(
|
||||
false,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
),
|
||||
joinableClausesIndexedTableLongKey,
|
||||
VirtualColumns.EMPTY,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
||||
hashJoinIndexedTableLongKeySegment = new HashJoinSegment(
|
||||
ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment),
|
||||
joinableClausesIndexedTableLonggKey,
|
||||
preAnalysisGroupIndexedLongKey
|
||||
joinableClausesIndexedTableLongKey,
|
||||
preAnalysisIndexedLongKey
|
||||
);
|
||||
|
||||
final Map<String, String> countryCodeToNameMap = JoinTestHelper.createCountryIsoCodeToNameLookup().getMap();
|
||||
|
|
|
@ -44,7 +44,9 @@ import java.util.Optional;
|
|||
*
|
||||
* <pre>
|
||||
*
|
||||
* Q <-- Possible outer query datasource(s) [may be multiple stacked]
|
||||
* Q <-- Possible query datasource(s) [may be none, or multiple stacked]
|
||||
* |
|
||||
* Q <-- Base query datasource, returned by {@link #getBaseQuery()} if it exists
|
||||
* |
|
||||
* J <-- Possible join tree, expected to be left-leaning
|
||||
* / \
|
||||
|
@ -77,13 +79,13 @@ public class DataSourceAnalysis
|
|||
private final DataSource dataSource;
|
||||
private final DataSource baseDataSource;
|
||||
@Nullable
|
||||
private final QuerySegmentSpec baseQuerySegmentSpec;
|
||||
private final Query<?> baseQuery;
|
||||
private final List<PreJoinableClause> preJoinableClauses;
|
||||
|
||||
private DataSourceAnalysis(
|
||||
DataSource dataSource,
|
||||
DataSource baseDataSource,
|
||||
@Nullable QuerySegmentSpec baseQuerySegmentSpec,
|
||||
@Nullable Query<?> baseQuery,
|
||||
List<PreJoinableClause> preJoinableClauses
|
||||
)
|
||||
{
|
||||
|
@ -95,33 +97,34 @@ public class DataSourceAnalysis
|
|||
|
||||
this.dataSource = dataSource;
|
||||
this.baseDataSource = baseDataSource;
|
||||
this.baseQuerySegmentSpec = baseQuerySegmentSpec;
|
||||
this.baseQuery = baseQuery;
|
||||
this.preJoinableClauses = preJoinableClauses;
|
||||
}
|
||||
|
||||
public static DataSourceAnalysis forDataSource(final DataSource dataSource)
|
||||
{
|
||||
// Strip outer queries, retaining querySegmentSpecs as we go down (lowest will become the 'baseQuerySegmentSpec').
|
||||
QuerySegmentSpec baseQuerySegmentSpec = null;
|
||||
Query<?> baseQuery = null;
|
||||
DataSource current = dataSource;
|
||||
|
||||
while (current instanceof QueryDataSource) {
|
||||
final Query<?> subQuery = ((QueryDataSource) current).getQuery();
|
||||
|
||||
if (!(subQuery instanceof BaseQuery)) {
|
||||
// All builtin query types are BaseQuery, so we only expect this with funky extension queries.
|
||||
// We must verify that the subQuery is a BaseQuery, because it is required to make "getBaseQuerySegmentSpec"
|
||||
// work properly. All builtin query types are BaseQuery, so we only expect this with funky extension queries.
|
||||
throw new IAE("Cannot analyze subquery of class[%s]", subQuery.getClass().getName());
|
||||
}
|
||||
|
||||
baseQuerySegmentSpec = ((BaseQuery<?>) subQuery).getQuerySegmentSpec();
|
||||
baseQuery = subQuery;
|
||||
current = subQuery.getDataSource();
|
||||
}
|
||||
|
||||
if (current instanceof JoinDataSource) {
|
||||
final Pair<DataSource, List<PreJoinableClause>> flattened = flattenJoin((JoinDataSource) current);
|
||||
return new DataSourceAnalysis(dataSource, flattened.lhs, baseQuerySegmentSpec, flattened.rhs);
|
||||
return new DataSourceAnalysis(dataSource, flattened.lhs, baseQuery, flattened.rhs);
|
||||
} else {
|
||||
return new DataSourceAnalysis(dataSource, current, baseQuerySegmentSpec, Collections.emptyList());
|
||||
return new DataSourceAnalysis(dataSource, current, baseQuery, Collections.emptyList());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -185,14 +188,30 @@ public class DataSourceAnalysis
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the bottommost (i.e. innermost) {@link Query} from a possible stack of outer queries at the root of
|
||||
* the datasource tree. This is the query that will be applied to the base datasource plus any joinables that might
|
||||
* be present.
|
||||
*
|
||||
* @return the query associated with the base datasource if {@link #isQuery()} is true, else empty
|
||||
*/
|
||||
public Optional<Query<?>> getBaseQuery()
|
||||
{
|
||||
return Optional.ofNullable(baseQuery);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link QuerySegmentSpec} that is associated with the base datasource, if any. This only happens
|
||||
* when there is an outer query datasource. In this case, the base querySegmentSpec is the one associated with the
|
||||
* innermost subquery.
|
||||
*
|
||||
* This {@link QuerySegmentSpec} is taken from the query returned by {@link #getBaseQuery()}.
|
||||
*
|
||||
* @return the query segment spec associated with the base datasource if {@link #isQuery()} is true, else empty
|
||||
*/
|
||||
public Optional<QuerySegmentSpec> getBaseQuerySegmentSpec()
|
||||
{
|
||||
return Optional.ofNullable(baseQuerySegmentSpec);
|
||||
return getBaseQuery().map(query -> ((BaseQuery<?>) query).getQuerySegmentSpec());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -240,7 +259,7 @@ public class DataSourceAnalysis
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns true if this datasource represents a subquery.
|
||||
* Returns true if this datasource represents a subquery (that is, whether it is a {@link QueryDataSource}).
|
||||
*/
|
||||
public boolean isQuery()
|
||||
{
|
||||
|
@ -257,16 +276,13 @@ public class DataSourceAnalysis
|
|||
return false;
|
||||
}
|
||||
DataSourceAnalysis that = (DataSourceAnalysis) o;
|
||||
return Objects.equals(dataSource, that.dataSource) &&
|
||||
Objects.equals(baseDataSource, that.baseDataSource) &&
|
||||
Objects.equals(baseQuerySegmentSpec, that.baseQuerySegmentSpec) &&
|
||||
Objects.equals(preJoinableClauses, that.preJoinableClauses);
|
||||
return Objects.equals(dataSource, that.dataSource);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(dataSource, baseDataSource, baseQuerySegmentSpec, preJoinableClauses);
|
||||
return Objects.hash(dataSource);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -275,8 +291,8 @@ public class DataSourceAnalysis
|
|||
return "DataSourceAnalysis{" +
|
||||
"dataSource=" + dataSource +
|
||||
", baseDataSource=" + baseDataSource +
|
||||
", baseQuerySegmentSpec=" + baseQuerySegmentSpec +
|
||||
", joinClauses=" + preJoinableClauses +
|
||||
", baseQuery=" + baseQuery +
|
||||
", preJoinableClauses=" + preJoinableClauses +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.druid.java.util.common.io.Closer;
|
|||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
@ -44,24 +44,23 @@ public class HashJoinSegment implements SegmentReference
|
|||
{
|
||||
private final SegmentReference baseSegment;
|
||||
private final List<JoinableClause> clauses;
|
||||
private final JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup;
|
||||
private final JoinFilterPreAnalysis joinFilterPreAnalysis;
|
||||
|
||||
/**
|
||||
* @param baseSegment The left-hand side base segment
|
||||
* @param clauses The right-hand side clauses. The caller is responsible for ensuring that there are no
|
||||
* duplicate prefixes or prefixes that shadow each other across the clauses
|
||||
* @param joinFilterPreAnalysisGroup Pre-analysis group that holds all of the JoinFilterPreAnalysis results within
|
||||
* the scope of a query
|
||||
* @param joinFilterPreAnalysis Pre-analysis for the query we expect to run on this segment
|
||||
*/
|
||||
public HashJoinSegment(
|
||||
SegmentReference baseSegment,
|
||||
List<JoinableClause> clauses,
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis
|
||||
)
|
||||
{
|
||||
this.baseSegment = baseSegment;
|
||||
this.clauses = clauses;
|
||||
this.joinFilterPreAnalysisGroup = joinFilterPreAnalysisGroup;
|
||||
this.joinFilterPreAnalysis = joinFilterPreAnalysis;
|
||||
|
||||
// Verify 'clauses' is nonempty (otherwise it's a waste to create this object, and the caller should know)
|
||||
if (clauses.isEmpty()) {
|
||||
|
@ -94,7 +93,7 @@ public class HashJoinSegment implements SegmentReference
|
|||
@Override
|
||||
public StorageAdapter asStorageAdapter()
|
||||
{
|
||||
return new HashJoinSegmentStorageAdapter(baseSegment.asStorageAdapter(), clauses, joinFilterPreAnalysisGroup);
|
||||
return new HashJoinSegmentStorageAdapter(baseSegment.asStorageAdapter(), clauses, joinFilterPreAnalysis);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.segment.join;
|
|||
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.java.util.common.guava.Sequences;
|
||||
|
@ -36,8 +37,8 @@ import org.apache.druid.segment.data.Indexed;
|
|||
import org.apache.druid.segment.data.ListIndexed;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterAnalyzer;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterSplit;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
@ -54,23 +55,22 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter
|
|||
{
|
||||
private final StorageAdapter baseAdapter;
|
||||
private final List<JoinableClause> clauses;
|
||||
private final JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup;
|
||||
private final JoinFilterPreAnalysis joinFilterPreAnalysis;
|
||||
|
||||
/**
|
||||
* @param baseAdapter A StorageAdapter for the left-hand side base segment
|
||||
* @param clauses The right-hand side clauses. The caller is responsible for ensuring that there are no
|
||||
* @param joinFilterPreAnalysisGroup Pre-analysis group that holds all of the JoinFilterPreAnalysis results within
|
||||
* the scope of a query
|
||||
* @param baseAdapter A StorageAdapter for the left-hand side base segment
|
||||
* @param clauses The right-hand side clauses. The caller is responsible for ensuring that there are no
|
||||
* @param joinFilterPreAnalysis Pre-analysis for the query we expect to run on this storage adapter
|
||||
*/
|
||||
HashJoinSegmentStorageAdapter(
|
||||
StorageAdapter baseAdapter,
|
||||
List<JoinableClause> clauses,
|
||||
final JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup
|
||||
final StorageAdapter baseAdapter,
|
||||
final List<JoinableClause> clauses,
|
||||
final JoinFilterPreAnalysis joinFilterPreAnalysis
|
||||
)
|
||||
{
|
||||
this.baseAdapter = baseAdapter;
|
||||
this.clauses = clauses;
|
||||
this.joinFilterPreAnalysisGroup = joinFilterPreAnalysisGroup;
|
||||
this.joinFilterPreAnalysis = joinFilterPreAnalysis;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -209,14 +209,24 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter
|
|||
@Nullable final QueryMetrics<?> queryMetrics
|
||||
)
|
||||
{
|
||||
JoinFilterPreAnalysis jfpa;
|
||||
if (joinFilterPreAnalysisGroup.isSingleLevelMode()) {
|
||||
jfpa = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
} else {
|
||||
jfpa = joinFilterPreAnalysisGroup.getAnalysis(
|
||||
filter,
|
||||
virtualColumns
|
||||
);
|
||||
// Filter pre-analysis key implied by the call to "makeCursors". We need to sanity-check that it matches
|
||||
// the actual pre-analysis that was done. Note: we can't infer a rewrite config from the "makeCursors" call (it
|
||||
// requires access to the query context) so we'll need to skip sanity-checking it, by re-using the one present
|
||||
// in the cached key.)
|
||||
final JoinFilterPreAnalysisKey keyIn =
|
||||
new JoinFilterPreAnalysisKey(
|
||||
joinFilterPreAnalysis.getKey().getRewriteConfig(),
|
||||
clauses,
|
||||
virtualColumns,
|
||||
filter
|
||||
);
|
||||
|
||||
final JoinFilterPreAnalysisKey keyCached = joinFilterPreAnalysis.getKey();
|
||||
|
||||
if (!keyIn.equals(keyCached)) {
|
||||
// It is a bug if this happens. We expect the comparison to be quick, because in the sane case, identical objects
|
||||
// will be used and therefore deep equality checks will be unnecessary.
|
||||
throw new ISE("Pre-analysis mismatch, cannot execute query");
|
||||
}
|
||||
|
||||
final List<VirtualColumn> preJoinVirtualColumns = new ArrayList<>();
|
||||
|
@ -228,7 +238,7 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter
|
|||
postJoinVirtualColumns
|
||||
);
|
||||
|
||||
JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter(jfpa);
|
||||
JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
preJoinVirtualColumns.addAll(joinFilterSplit.getPushDownVirtualColumns());
|
||||
|
||||
// Soon, we will need a way to push filters past a join when possible. This could potentially be done right here
|
||||
|
|
|
@ -20,20 +20,19 @@
|
|||
package org.apache.druid.segment.join;
|
||||
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.JoinDataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.planning.PreJoinableClause;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterAnalyzer;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey;
|
||||
import org.apache.druid.segment.join.filter.JoinableClauses;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
import org.apache.druid.utils.JvmUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
@ -74,19 +73,21 @@ public class Joinables
|
|||
/**
|
||||
* Creates a Function that maps base segments to {@link HashJoinSegment} if needed (i.e. if the number of join
|
||||
* clauses is > 0). If mapping is not needed, this method will return {@link Function#identity()}.
|
||||
* @param clauses Pre-joinable clauses
|
||||
* @param joinableFactory Factory for joinables
|
||||
* @param cpuTimeAccumulator An accumulator that we will add CPU nanos to; this is part of the function to encourage
|
||||
* callers to remember to track metrics on CPU time required for creation of Joinables
|
||||
* @param joinFilterRewriteConfig Configuration options for the join filter rewrites
|
||||
* @param query The query being processed
|
||||
*
|
||||
* @param clauses Pre-joinable clauses
|
||||
* @param joinableFactory Factory for joinables
|
||||
* @param cpuTimeAccumulator An accumulator that we will add CPU nanos to; this is part of the function to encourage
|
||||
* callers to remember to track metrics on CPU time required for creation of Joinables
|
||||
* @param query The query that will be run on the mapped segments. Usually this should be
|
||||
* {@code analysis.getBaseQuery().orElse(query)}, where "analysis" is a
|
||||
* {@link org.apache.druid.query.planning.DataSourceAnalysis} and "query" is the original
|
||||
* query from the end user.
|
||||
*/
|
||||
public static Function<SegmentReference, SegmentReference> createSegmentMapFn(
|
||||
final List<PreJoinableClause> clauses,
|
||||
final JoinableFactory joinableFactory,
|
||||
final AtomicLong cpuTimeAccumulator,
|
||||
final JoinFilterRewriteConfig joinFilterRewriteConfig,
|
||||
final Query query
|
||||
final Query<?> query
|
||||
)
|
||||
{
|
||||
// compute column correlations here and RHS correlated values
|
||||
|
@ -97,85 +98,26 @@ public class Joinables
|
|||
return Function.identity();
|
||||
} else {
|
||||
final JoinableClauses joinableClauses = JoinableClauses.createClauses(clauses, joinableFactory);
|
||||
|
||||
List<Query> joinQueryLevels = new ArrayList<>();
|
||||
Joinables.gatherAllJoinQueryLevels(query, joinQueryLevels);
|
||||
|
||||
final JoinFilterPreAnalysisGroup preAnalysisGroup = new JoinFilterPreAnalysisGroup(
|
||||
joinFilterRewriteConfig,
|
||||
joinQueryLevels.size() <= 1 // use single-level mode if there's one or fewer query levels with joins
|
||||
final JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis(
|
||||
new JoinFilterPreAnalysisKey(
|
||||
JoinFilterRewriteConfig.forQuery(query),
|
||||
joinableClauses.getJoinableClauses(),
|
||||
query.getVirtualColumns(),
|
||||
Filters.toFilter(query.getFilter())
|
||||
)
|
||||
);
|
||||
|
||||
for (Query joinQuery : joinQueryLevels) {
|
||||
preAnalysisGroup.computeJoinFilterPreAnalysisIfAbsent(
|
||||
joinQuery.getFilter() == null ? null : joinQuery.getFilter().toFilter(),
|
||||
joinableClauses.getJoinableClauses(),
|
||||
joinQuery.getVirtualColumns()
|
||||
);
|
||||
}
|
||||
|
||||
return baseSegment -> new HashJoinSegment(baseSegment, joinableClauses.getJoinableClauses(), preAnalysisGroup);
|
||||
return baseSegment ->
|
||||
new HashJoinSegment(
|
||||
baseSegment,
|
||||
joinableClauses.getJoinableClauses(),
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Walks a query and its subqueries, finding any queries that read from a JoinDatasource,
|
||||
* and adding them to a list provided by the caller.
|
||||
*
|
||||
* @param currentLevelQuery The query to analyze
|
||||
* @param allJoinQueryLevels A mutable list provided by the caller.
|
||||
*/
|
||||
public static void gatherAllJoinQueryLevels(Query currentLevelQuery, List<Query> allJoinQueryLevels)
|
||||
{
|
||||
DataSource currentDatasource = currentLevelQuery.getDataSource();
|
||||
if (currentDatasource instanceof QueryDataSource) {
|
||||
gatherAllJoinQueryLevels(
|
||||
((QueryDataSource) currentDatasource).getQuery(),
|
||||
allJoinQueryLevels
|
||||
);
|
||||
}
|
||||
if (currentDatasource instanceof JoinDataSource) {
|
||||
allJoinQueryLevels.add(currentLevelQuery);
|
||||
gatherAllJoinQueryLevelsJoinDatasourceHelper(
|
||||
(JoinDataSource) currentDatasource,
|
||||
allJoinQueryLevels
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
private static void gatherAllJoinQueryLevelsJoinDatasourceHelper(
|
||||
JoinDataSource joinDatasource,
|
||||
List<Query> allJoinQueryLevels
|
||||
)
|
||||
{
|
||||
if (joinDatasource.getLeft() instanceof QueryDataSource) {
|
||||
gatherAllJoinQueryLevels(
|
||||
((QueryDataSource) joinDatasource.getLeft()).getQuery(),
|
||||
allJoinQueryLevels
|
||||
);
|
||||
}
|
||||
if (joinDatasource.getLeft() instanceof JoinDataSource) {
|
||||
gatherAllJoinQueryLevelsJoinDatasourceHelper(
|
||||
(JoinDataSource) joinDatasource.getLeft(),
|
||||
allJoinQueryLevels
|
||||
);
|
||||
}
|
||||
if (joinDatasource.getRight() instanceof QueryDataSource) {
|
||||
gatherAllJoinQueryLevels(
|
||||
((QueryDataSource) joinDatasource.getRight()).getQuery(),
|
||||
allJoinQueryLevels
|
||||
);
|
||||
}
|
||||
if (joinDatasource.getRight() instanceof JoinDataSource) {
|
||||
gatherAllJoinQueryLevelsJoinDatasourceHelper(
|
||||
(JoinDataSource) joinDatasource.getRight(),
|
||||
allJoinQueryLevels
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if any prefixes in the provided list duplicate or shadow each other.
|
||||
*
|
||||
|
|
|
@ -26,13 +26,11 @@ import org.apache.druid.math.expr.Expr;
|
|||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.InDimFilter;
|
||||
import org.apache.druid.segment.VirtualColumn;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.filter.FalseFilter;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.filter.OrFilter;
|
||||
import org.apache.druid.segment.filter.SelectorFilter;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
@ -47,27 +45,27 @@ import java.util.Set;
|
|||
/**
|
||||
* When there is a filter in a join query, we can sometimes improve performance by applying parts of the filter
|
||||
* when we first read from the base table instead of after the join.
|
||||
*
|
||||
*
|
||||
* The first step of the filter splitting is to convert the filter into
|
||||
* https://en.wikipedia.org/wiki/Conjunctive_normal_form (an AND of ORs). This allows us to consider each
|
||||
* OR clause independently as a candidate for filter push down to the base table.
|
||||
*
|
||||
*
|
||||
* A filter clause can be pushed down if it meets one of the following conditions:
|
||||
* - The filter only applies to columns from the base table
|
||||
* - The filter applies to columns from the join table, and we determine that the filter can be rewritten
|
||||
* into a filter on columns from the base table
|
||||
*
|
||||
*
|
||||
* For the second case, where we rewrite filter clauses, the rewritten clause can be less selective than the original,
|
||||
* so we preserve the original clause in the post-join filtering phase.
|
||||
*
|
||||
*
|
||||
* The starting point for join analysis is the {@link #computeJoinFilterPreAnalysis} method. This method should be
|
||||
* called before performing any per-segment join query work. This method converts the query filter into
|
||||
* conjunctive normal form, and splits the CNF clauses into a portion that only references base table columns and
|
||||
* a portion that references join table columns. For the filter clauses that apply to join table columns, the
|
||||
* pre-analysis step computes the information necessary for rewriting such filters into filters on base table columns.
|
||||
*
|
||||
*
|
||||
* The result of this pre-analysis method should be passed into the next step of join filter analysis, described below.
|
||||
*
|
||||
*
|
||||
* The {@link #splitFilter(JoinFilterPreAnalysis)} method takes the pre-analysis result and optionally applies the
|
||||
* filter rewrite and push down operations on a per-segment level.
|
||||
*/
|
||||
|
@ -80,33 +78,29 @@ public class JoinFilterAnalyzer
|
|||
* where we convert the query filter (if any) into conjunctive normal form and then
|
||||
* determine the structure of RHS filter rewrites (if any), since this information is shared across all
|
||||
* per-segment operations.
|
||||
*
|
||||
*
|
||||
* See {@link JoinFilterPreAnalysis} for details on the result of this pre-analysis step.
|
||||
*
|
||||
* @param joinableClauses The joinable clauses from the query
|
||||
* @param virtualColumns The virtual columns from the query
|
||||
* @param originalFilter The original filter from the query
|
||||
* @param joinFilterRewriteConfig Configuration options for the join rewrites
|
||||
* @param key All the information needed to pre-analyze a filter
|
||||
*
|
||||
* @return A JoinFilterPreAnalysis containing information determined in this pre-analysis step.
|
||||
*/
|
||||
public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis(
|
||||
JoinableClauses joinableClauses,
|
||||
VirtualColumns virtualColumns,
|
||||
Filter originalFilter,
|
||||
JoinFilterRewriteConfig joinFilterRewriteConfig
|
||||
)
|
||||
public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis(final JoinFilterPreAnalysisKey key)
|
||||
{
|
||||
final List<VirtualColumn> preJoinVirtualColumns = new ArrayList<>();
|
||||
final List<VirtualColumn> postJoinVirtualColumns = new ArrayList<>();
|
||||
|
||||
joinableClauses.splitVirtualColumns(virtualColumns, preJoinVirtualColumns, postJoinVirtualColumns);
|
||||
JoinFilterPreAnalysis.Builder preAnalysisBuilder =
|
||||
new JoinFilterPreAnalysis.Builder(joinFilterRewriteConfig, joinableClauses, originalFilter, postJoinVirtualColumns);
|
||||
if (originalFilter == null || !joinFilterRewriteConfig.isEnableFilterPushDown()) {
|
||||
final JoinableClauses joinableClauses = JoinableClauses.fromList(key.getJoinableClauses());
|
||||
joinableClauses.splitVirtualColumns(key.getVirtualColumns(), preJoinVirtualColumns, postJoinVirtualColumns);
|
||||
|
||||
final JoinFilterPreAnalysis.Builder preAnalysisBuilder =
|
||||
new JoinFilterPreAnalysis.Builder(key, postJoinVirtualColumns);
|
||||
|
||||
if (key.getFilter() == null || !key.getRewriteConfig().isEnableFilterPushDown()) {
|
||||
return preAnalysisBuilder.build();
|
||||
}
|
||||
|
||||
Set<Filter> normalizedOrClauses = Filters.toNormalizedOrClauses(originalFilter);
|
||||
Set<Filter> normalizedOrClauses = Filters.toNormalizedOrClauses(key.getFilter());
|
||||
|
||||
List<Filter> normalizedBaseTableClauses = new ArrayList<>();
|
||||
List<Filter> normalizedJoinTableClauses = new ArrayList<>();
|
||||
|
@ -125,7 +119,7 @@ public class JoinFilterAnalyzer
|
|||
preAnalysisBuilder
|
||||
.withNormalizedBaseTableClauses(normalizedBaseTableClauses)
|
||||
.withNormalizedJoinTableClauses(normalizedJoinTableClauses);
|
||||
if (!joinFilterRewriteConfig.isEnableFilterRewrite()) {
|
||||
if (!key.getRewriteConfig().isEnableFilterRewrite()) {
|
||||
return preAnalysisBuilder.build();
|
||||
}
|
||||
|
||||
|
@ -136,12 +130,11 @@ public class JoinFilterAnalyzer
|
|||
normalizedJoinTableClauses,
|
||||
equiconditions,
|
||||
joinableClauses,
|
||||
joinFilterRewriteConfig.isEnableRewriteValueColumnFilters(),
|
||||
joinFilterRewriteConfig.getFilterRewriteMaxSize()
|
||||
key.getRewriteConfig().isEnableRewriteValueColumnFilters(),
|
||||
key.getRewriteConfig().getFilterRewriteMaxSize()
|
||||
);
|
||||
|
||||
return preAnalysisBuilder.withCorrelations(correlations)
|
||||
.build();
|
||||
return preAnalysisBuilder.withCorrelations(correlations).build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -201,16 +194,15 @@ public class JoinFilterAnalyzer
|
|||
* Analyze a filter clause from a filter that is in conjunctive normal form (AND of ORs).
|
||||
* The clause is expected to be an OR filter or a leaf filter.
|
||||
*
|
||||
* @param filterClause Individual filter clause (an OR filter or a leaf filter) from a filter that is in CNF
|
||||
* @param joinFilterPreAnalysis The pre-analysis computed by {@link #computeJoinFilterPreAnalysis)}
|
||||
* @param pushDownVirtualColumnsForLhsExprs Used when there are LHS expressions in the join equiconditions.
|
||||
* If we rewrite an RHS filter such that it applies to the LHS expression instead,
|
||||
* because the expression existed only in the equicondition, we must create a virtual column
|
||||
* on the LHS with the same expression in order to apply the filter.
|
||||
* The specific rewriting methods such as {@link #rewriteSelectorFilter} will use this
|
||||
* as a cache for virtual columns that they need to created, keyed by the expression, so that
|
||||
* they can avoid creating redundant virtual columns.
|
||||
*
|
||||
* @param filterClause Individual filter clause (an OR filter or a leaf filter) from a filter that is in CNF
|
||||
* @param joinFilterPreAnalysis The pre-analysis computed by {@link #computeJoinFilterPreAnalysis)}
|
||||
* @param pushDownVirtualColumnsForLhsExprs Used when there are LHS expressions in the join equiconditions.
|
||||
* If we rewrite an RHS filter such that it applies to the LHS expression instead,
|
||||
* because the expression existed only in the equicondition, we must create a virtual column
|
||||
* on the LHS with the same expression in order to apply the filter.
|
||||
* The specific rewriting methods such as {@link #rewriteSelectorFilter} will use this
|
||||
* as a cache for virtual columns that they need to created, keyed by the expression, so that
|
||||
* they can avoid creating redundant virtual columns.
|
||||
*
|
||||
* @return a JoinFilterAnalysis that contains a possible filter rewrite and information on how to handle the filter.
|
||||
*/
|
||||
|
@ -325,9 +317,10 @@ public class JoinFilterAnalyzer
|
|||
* Potentially rewrite the subfilters of an OR filter so that the whole OR filter can be pushed down to
|
||||
* the base table.
|
||||
*
|
||||
* @param orFilter OrFilter to be rewritten
|
||||
* @param joinFilterPreAnalysis The pre-analysis computed by {@link #computeJoinFilterPreAnalysis)}
|
||||
* @param orFilter OrFilter to be rewritten
|
||||
* @param joinFilterPreAnalysis The pre-analysis computed by {@link #computeJoinFilterPreAnalysis)}
|
||||
* @param pushDownVirtualColumnsForLhsExprs See comments on {@link #analyzeJoinFilterClause}
|
||||
*
|
||||
* @return A JoinFilterAnalysis indicating how to handle the potentially rewritten filter
|
||||
*/
|
||||
private static JoinFilterAnalysis rewriteOrFilter(
|
||||
|
@ -383,9 +376,10 @@ public class JoinFilterAnalyzer
|
|||
/**
|
||||
* Rewrites a selector filter on a join table into an IN filter on the base table.
|
||||
*
|
||||
* @param selectorFilter SelectorFilter to be rewritten
|
||||
* @param joinFilterPreAnalysis The pre-analysis computed by {@link #computeJoinFilterPreAnalysis)}
|
||||
* @param selectorFilter SelectorFilter to be rewritten
|
||||
* @param joinFilterPreAnalysis The pre-analysis computed by {@link #computeJoinFilterPreAnalysis)}
|
||||
* @param pushDownVirtualColumnsForLhsExprs See comments on {@link #analyzeJoinFilterClause}
|
||||
*
|
||||
* @return A JoinFilterAnalysis that indicates how to handle the potentially rewritten filter
|
||||
*/
|
||||
private static JoinFilterAnalysis rewriteSelectorFilter(
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.druid.query.filter.Filter;
|
|||
import org.apache.druid.segment.VirtualColumn;
|
||||
import org.apache.druid.segment.join.Equality;
|
||||
import org.apache.druid.segment.join.JoinableClause;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -47,44 +46,44 @@ import java.util.Set;
|
|||
*/
|
||||
public class JoinFilterPreAnalysis
|
||||
{
|
||||
private final JoinableClauses joinableClauses;
|
||||
private final Filter originalFilter;
|
||||
private final JoinFilterPreAnalysisKey key;
|
||||
private final List<Filter> normalizedBaseTableClauses;
|
||||
private final List<Filter> normalizedJoinTableClauses;
|
||||
private final JoinFilterCorrelations correlations;
|
||||
private final List<VirtualColumn> postJoinVirtualColumns;
|
||||
private final Equiconditions equiconditions;
|
||||
private final JoinFilterRewriteConfig rewriteConfig;
|
||||
|
||||
private JoinFilterPreAnalysis(
|
||||
final JoinableClauses joinableClauses,
|
||||
final Filter originalFilter,
|
||||
final JoinFilterPreAnalysisKey key,
|
||||
final List<VirtualColumn> postJoinVirtualColumns,
|
||||
final List<Filter> normalizedBaseTableClauses,
|
||||
final List<Filter> normalizedJoinTableClauses,
|
||||
JoinFilterCorrelations correlations,
|
||||
final Equiconditions equiconditions,
|
||||
final JoinFilterRewriteConfig rewriteConfig
|
||||
final JoinFilterCorrelations correlations,
|
||||
final Equiconditions equiconditions
|
||||
)
|
||||
{
|
||||
this.joinableClauses = joinableClauses;
|
||||
this.originalFilter = originalFilter;
|
||||
this.key = key;
|
||||
this.postJoinVirtualColumns = postJoinVirtualColumns;
|
||||
this.normalizedBaseTableClauses = normalizedBaseTableClauses;
|
||||
this.normalizedJoinTableClauses = normalizedJoinTableClauses;
|
||||
this.correlations = correlations;
|
||||
this.rewriteConfig = rewriteConfig;
|
||||
this.equiconditions = equiconditions;
|
||||
}
|
||||
|
||||
public JoinFilterPreAnalysisKey getKey()
|
||||
{
|
||||
return key;
|
||||
}
|
||||
|
||||
public JoinableClauses getJoinableClauses()
|
||||
{
|
||||
return joinableClauses;
|
||||
return JoinableClauses.fromList(key.getJoinableClauses());
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public Filter getOriginalFilter()
|
||||
{
|
||||
return originalFilter;
|
||||
return key.getFilter();
|
||||
}
|
||||
|
||||
public List<VirtualColumn> getPostJoinVirtualColumns()
|
||||
|
@ -114,12 +113,12 @@ public class JoinFilterPreAnalysis
|
|||
|
||||
public boolean isEnableFilterPushDown()
|
||||
{
|
||||
return rewriteConfig.isEnableFilterPushDown();
|
||||
return key.getRewriteConfig().isEnableFilterPushDown();
|
||||
}
|
||||
|
||||
public boolean isEnableFilterRewrite()
|
||||
{
|
||||
return rewriteConfig.isEnableFilterRewrite();
|
||||
return key.getRewriteConfig().isEnableFilterRewrite();
|
||||
}
|
||||
|
||||
public Equiconditions getEquiconditions()
|
||||
|
@ -132,25 +131,25 @@ public class JoinFilterPreAnalysis
|
|||
*/
|
||||
public static class Builder
|
||||
{
|
||||
@Nonnull private final JoinFilterRewriteConfig rewriteConfig;
|
||||
@Nonnull private final JoinableClauses joinableClauses;
|
||||
@Nullable private final Filter originalFilter;
|
||||
@Nullable private List<Filter> normalizedBaseTableClauses;
|
||||
@Nullable private List<Filter> normalizedJoinTableClauses;
|
||||
@Nullable private JoinFilterCorrelations correlations;
|
||||
@Nonnull private final List<VirtualColumn> postJoinVirtualColumns;
|
||||
@Nonnull private Equiconditions equiconditions = new Equiconditions(Collections.emptyMap());
|
||||
@Nonnull
|
||||
private final JoinFilterPreAnalysisKey key;
|
||||
@Nullable
|
||||
private List<Filter> normalizedBaseTableClauses;
|
||||
@Nullable
|
||||
private List<Filter> normalizedJoinTableClauses;
|
||||
@Nullable
|
||||
private JoinFilterCorrelations correlations;
|
||||
@Nonnull
|
||||
private final List<VirtualColumn> postJoinVirtualColumns;
|
||||
@Nonnull
|
||||
private Equiconditions equiconditions = new Equiconditions(Collections.emptyMap());
|
||||
|
||||
public Builder(
|
||||
@Nonnull JoinFilterRewriteConfig rewriteConfig,
|
||||
@Nonnull JoinableClauses joinableClauses,
|
||||
@Nullable Filter originalFilter,
|
||||
@Nonnull JoinFilterPreAnalysisKey key,
|
||||
@Nonnull List<VirtualColumn> postJoinVirtualColumns
|
||||
)
|
||||
{
|
||||
this.rewriteConfig = rewriteConfig;
|
||||
this.joinableClauses = joinableClauses;
|
||||
this.originalFilter = originalFilter;
|
||||
this.key = key;
|
||||
this.postJoinVirtualColumns = postJoinVirtualColumns;
|
||||
}
|
||||
|
||||
|
@ -177,7 +176,7 @@ public class JoinFilterPreAnalysis
|
|||
public Equiconditions computeEquiconditionsFromJoinableClauses()
|
||||
{
|
||||
Map<String, Set<Expr>> equiconditionsMap = new HashMap<>();
|
||||
for (JoinableClause clause : joinableClauses.getJoinableClauses()) {
|
||||
for (JoinableClause clause : key.getJoinableClauses()) {
|
||||
for (Equality equality : clause.getCondition().getEquiConditions()) {
|
||||
Set<Expr> exprsForRhs = equiconditionsMap.computeIfAbsent(
|
||||
clause.getPrefix() + equality.getRightColumn(),
|
||||
|
@ -193,14 +192,12 @@ public class JoinFilterPreAnalysis
|
|||
public JoinFilterPreAnalysis build()
|
||||
{
|
||||
return new JoinFilterPreAnalysis(
|
||||
joinableClauses,
|
||||
originalFilter,
|
||||
key,
|
||||
postJoinVirtualColumns,
|
||||
normalizedBaseTableClauses,
|
||||
normalizedJoinTableClauses,
|
||||
correlations,
|
||||
equiconditions,
|
||||
rewriteConfig
|
||||
equiconditions
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,97 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.join.filter;
|
||||
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
import org.apache.druid.segment.join.JoinableClause;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* All the information that is required to generate a {@link JoinFilterPreAnalysis}.
|
||||
*/
|
||||
public class JoinFilterPreAnalysisKey
|
||||
{
|
||||
private final JoinFilterRewriteConfig config;
|
||||
private final List<JoinableClause> clauses;
|
||||
private final VirtualColumns virtualColumns;
|
||||
|
||||
@Nullable
|
||||
private final Filter filter;
|
||||
|
||||
public JoinFilterPreAnalysisKey(
|
||||
final JoinFilterRewriteConfig config,
|
||||
final List<JoinableClause> clauses,
|
||||
final VirtualColumns virtualColumns,
|
||||
@Nullable final Filter filter
|
||||
)
|
||||
{
|
||||
this.config = config;
|
||||
this.clauses = clauses;
|
||||
this.virtualColumns = virtualColumns;
|
||||
this.filter = filter;
|
||||
}
|
||||
|
||||
public JoinFilterRewriteConfig getRewriteConfig()
|
||||
{
|
||||
return config;
|
||||
}
|
||||
|
||||
public List<JoinableClause> getJoinableClauses()
|
||||
{
|
||||
return clauses;
|
||||
}
|
||||
|
||||
public VirtualColumns getVirtualColumns()
|
||||
{
|
||||
return virtualColumns;
|
||||
}
|
||||
|
||||
public Filter getFilter()
|
||||
{
|
||||
return filter;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
JoinFilterPreAnalysisKey that = (JoinFilterPreAnalysisKey) o;
|
||||
return Objects.equals(config, that.config) &&
|
||||
Objects.equals(clauses, that.clauses) &&
|
||||
Objects.equals(virtualColumns, that.virtualColumns) &&
|
||||
Objects.equals(filter, that.filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(config, clauses, virtualColumns, filter);
|
||||
}
|
||||
}
|
|
@ -38,18 +38,22 @@ import java.util.stream.Collectors;
|
|||
|
||||
public class JoinableClauses
|
||||
{
|
||||
@Nonnull private final List<JoinableClause> joinableClauses;
|
||||
@Nonnull
|
||||
private final List<JoinableClause> joinableClauses;
|
||||
|
||||
/**
|
||||
* Builds a list of {@link JoinableClause} corresponding to a list of {@link PreJoinableClause}. This will call
|
||||
* {@link JoinableFactory#build} on each one and therefore may be an expensive operation.
|
||||
*/
|
||||
public static JoinableClauses createClauses(final List<PreJoinableClause> clauses, final JoinableFactory joinableFactory)
|
||||
public static JoinableClauses createClauses(
|
||||
final List<PreJoinableClause> preClauses,
|
||||
final JoinableFactory joinableFactory
|
||||
)
|
||||
{
|
||||
// Since building a JoinableClause can be expensive, check for prefix conflicts before building
|
||||
checkPreJoinableClausesForDuplicatesAndShadowing(clauses);
|
||||
checkPreJoinableClausesForDuplicatesAndShadowing(preClauses);
|
||||
|
||||
List<JoinableClause> joinableClauses = clauses.stream().map(preJoinableClause -> {
|
||||
List<JoinableClause> joinableClauses = preClauses.stream().map(preJoinableClause -> {
|
||||
final Optional<Joinable> joinable = joinableFactory.build(
|
||||
preJoinableClause.getDataSource(),
|
||||
preJoinableClause.getCondition()
|
||||
|
@ -65,6 +69,14 @@ public class JoinableClauses
|
|||
return new JoinableClauses(joinableClauses);
|
||||
}
|
||||
|
||||
/**
|
||||
* Wraps the provided list of pre-built {@link JoinableClause}. This is an inexpensive operation.
|
||||
*/
|
||||
public static JoinableClauses fromList(final List<JoinableClause> clauses)
|
||||
{
|
||||
return new JoinableClauses(clauses);
|
||||
}
|
||||
|
||||
private JoinableClauses(@Nonnull List<JoinableClause> joinableClauses)
|
||||
{
|
||||
this.joinableClauses = joinableClauses;
|
||||
|
@ -128,9 +140,4 @@ public class JoinableClauses
|
|||
|
||||
Joinables.checkPrefixesForDuplicatesAndShadowing(prefixes);
|
||||
}
|
||||
|
||||
public static JoinableClauses fromList(List<JoinableClause> clauses)
|
||||
{
|
||||
return new JoinableClauses(clauses);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,149 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.join.filter.rewrite;
|
||||
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
import org.apache.druid.segment.join.JoinableClause;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterAnalyzer;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
|
||||
import org.apache.druid.segment.join.filter.JoinableClauses;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* A JoinFilterPreAnalysisGroup holds all of the JoinFilterPreAnalysis objects for a given query and
|
||||
* also stores the per-query parameters that control the filter rewrite operations (from the query context).
|
||||
*
|
||||
* The analyses map is keyed by (Filter, JoinableClause list, VirtualColumns): each Filter in the map belongs to a
|
||||
* separate level of query (e.g. outer query, subquery level 1, etc.)
|
||||
*
|
||||
* If there is only a single Filter, then this class does not use the analyses map, instead of using a single reference
|
||||
* for efficiency reasons.
|
||||
*/
|
||||
public class JoinFilterPreAnalysisGroup
|
||||
{
|
||||
private final JoinFilterRewriteConfig joinFilterRewriteConfig;
|
||||
private final Map<JoinFilterPreAnalysisGroupKey, JoinFilterPreAnalysis> analyses;
|
||||
private final boolean isSingleLevelMode;
|
||||
|
||||
/**
|
||||
* Hashing and comparing filters can be expensive for large filters, so if we're only dealing with
|
||||
* a single level of join query, then we can be more efficient by using a single reference instead of a map.
|
||||
*/
|
||||
private JoinFilterPreAnalysis preAnalysisForSingleLevelMode;
|
||||
|
||||
public JoinFilterPreAnalysisGroup(
|
||||
JoinFilterRewriteConfig joinFilterRewriteConfig,
|
||||
boolean isSingleLevelMode
|
||||
)
|
||||
{
|
||||
this.joinFilterRewriteConfig = joinFilterRewriteConfig;
|
||||
this.analyses = new HashMap<>();
|
||||
this.isSingleLevelMode = isSingleLevelMode;
|
||||
}
|
||||
|
||||
public boolean isSingleLevelMode()
|
||||
{
|
||||
return isSingleLevelMode;
|
||||
}
|
||||
|
||||
public JoinFilterPreAnalysis computeJoinFilterPreAnalysisIfAbsent(
|
||||
Filter filter,
|
||||
List<JoinableClause> clauses,
|
||||
VirtualColumns virtualColumns
|
||||
)
|
||||
{
|
||||
if (isSingleLevelMode) {
|
||||
preAnalysisForSingleLevelMode = JoinFilterAnalyzer.computeJoinFilterPreAnalysis(
|
||||
JoinableClauses.fromList(clauses),
|
||||
virtualColumns,
|
||||
filter,
|
||||
joinFilterRewriteConfig
|
||||
);
|
||||
return preAnalysisForSingleLevelMode;
|
||||
}
|
||||
|
||||
JoinFilterPreAnalysisGroupKey key = new JoinFilterPreAnalysisGroupKey(filter, virtualColumns);
|
||||
return analyses.computeIfAbsent(
|
||||
key,
|
||||
(groupKey) -> {
|
||||
return JoinFilterAnalyzer.computeJoinFilterPreAnalysis(
|
||||
JoinableClauses.fromList(clauses),
|
||||
virtualColumns,
|
||||
filter,
|
||||
joinFilterRewriteConfig
|
||||
);
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
public JoinFilterPreAnalysis getAnalysis(
|
||||
Filter filter,
|
||||
VirtualColumns virtualColumns
|
||||
)
|
||||
{
|
||||
JoinFilterPreAnalysisGroupKey key = new JoinFilterPreAnalysisGroupKey(filter, virtualColumns);
|
||||
return analyses.get(key);
|
||||
}
|
||||
|
||||
public JoinFilterPreAnalysis getPreAnalysisForSingleLevelMode()
|
||||
{
|
||||
return preAnalysisForSingleLevelMode;
|
||||
}
|
||||
|
||||
public static class JoinFilterPreAnalysisGroupKey
|
||||
{
|
||||
private final Filter filter;
|
||||
private final VirtualColumns virtualColumns;
|
||||
|
||||
public JoinFilterPreAnalysisGroupKey(
|
||||
Filter filter,
|
||||
VirtualColumns virtualColumns
|
||||
)
|
||||
{
|
||||
this.filter = filter;
|
||||
this.virtualColumns = virtualColumns;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
JoinFilterPreAnalysisGroupKey that = (JoinFilterPreAnalysisGroupKey) o;
|
||||
return Objects.equals(filter, that.filter) &&
|
||||
Objects.equals(virtualColumns, that.virtualColumns);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(filter, virtualColumns);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -19,6 +19,11 @@
|
|||
|
||||
package org.apache.druid.segment.join.filter.rewrite;
|
||||
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* A config class that holds properties that control how join filter rewrites behave.
|
||||
*/
|
||||
|
@ -61,6 +66,16 @@ public class JoinFilterRewriteConfig
|
|||
this.filterRewriteMaxSize = filterRewriteMaxSize;
|
||||
}
|
||||
|
||||
public static JoinFilterRewriteConfig forQuery(final Query<?> query)
|
||||
{
|
||||
return new JoinFilterRewriteConfig(
|
||||
QueryContexts.getEnableJoinFilterPushDown(query),
|
||||
QueryContexts.getEnableJoinFilterRewrite(query),
|
||||
QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query),
|
||||
QueryContexts.getJoinFilterRewriteMaxSize(query)
|
||||
);
|
||||
}
|
||||
|
||||
public boolean isEnableFilterPushDown()
|
||||
{
|
||||
return enableFilterPushDown;
|
||||
|
@ -81,4 +96,30 @@ public class JoinFilterRewriteConfig
|
|||
return filterRewriteMaxSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
JoinFilterRewriteConfig that = (JoinFilterRewriteConfig) o;
|
||||
return enableFilterPushDown == that.enableFilterPushDown &&
|
||||
enableFilterRewrite == that.enableFilterRewrite &&
|
||||
enableRewriteValueColumnFilters == that.enableRewriteValueColumnFilters &&
|
||||
filterRewriteMaxSize == that.filterRewriteMaxSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(
|
||||
enableFilterPushDown,
|
||||
enableFilterRewrite,
|
||||
enableRewriteValueColumnFilters,
|
||||
filterRewriteMaxSize
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -69,6 +69,7 @@ public class DataSourceAnalysisTest
|
|||
Assert.assertEquals(TABLE_FOO, analysis.getDataSource());
|
||||
Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource());
|
||||
Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuery());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec());
|
||||
Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses());
|
||||
}
|
||||
|
@ -86,6 +87,7 @@ public class DataSourceAnalysisTest
|
|||
Assert.assertEquals(unionDataSource, analysis.getDataSource());
|
||||
Assert.assertEquals(unionDataSource, analysis.getBaseDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuery());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec());
|
||||
Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses());
|
||||
}
|
||||
|
@ -103,6 +105,7 @@ public class DataSourceAnalysisTest
|
|||
Assert.assertEquals(queryDataSource, analysis.getDataSource());
|
||||
Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource());
|
||||
Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource());
|
||||
Assert.assertEquals(Optional.of(queryDataSource.getQuery()), analysis.getBaseQuery());
|
||||
Assert.assertEquals(
|
||||
Optional.of(new MultipleIntervalSegmentSpec(MILLENIUM_INTERVALS)),
|
||||
analysis.getBaseQuerySegmentSpec()
|
||||
|
@ -124,6 +127,7 @@ public class DataSourceAnalysisTest
|
|||
Assert.assertEquals(queryDataSource, analysis.getDataSource());
|
||||
Assert.assertEquals(unionDataSource, analysis.getBaseDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource());
|
||||
Assert.assertEquals(Optional.of(queryDataSource.getQuery()), analysis.getBaseQuery());
|
||||
Assert.assertEquals(
|
||||
Optional.of(new MultipleIntervalSegmentSpec(MILLENIUM_INTERVALS)),
|
||||
analysis.getBaseQuerySegmentSpec()
|
||||
|
@ -143,6 +147,7 @@ public class DataSourceAnalysisTest
|
|||
Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getDataSource());
|
||||
Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getBaseDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuery());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec());
|
||||
Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses());
|
||||
}
|
||||
|
@ -160,6 +165,7 @@ public class DataSourceAnalysisTest
|
|||
Assert.assertEquals(queryDataSource, analysis.getDataSource());
|
||||
Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getBaseDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource());
|
||||
Assert.assertEquals(Optional.of(queryDataSource.getQuery()), analysis.getBaseQuery());
|
||||
Assert.assertEquals(
|
||||
Optional.of(new MultipleIntervalSegmentSpec(MILLENIUM_INTERVALS)),
|
||||
analysis.getBaseQuerySegmentSpec()
|
||||
|
@ -179,6 +185,7 @@ public class DataSourceAnalysisTest
|
|||
Assert.assertEquals(INLINE, analysis.getDataSource());
|
||||
Assert.assertEquals(INLINE, analysis.getBaseDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuery());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec());
|
||||
Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses());
|
||||
}
|
||||
|
@ -216,6 +223,7 @@ public class DataSourceAnalysisTest
|
|||
Assert.assertEquals(joinDataSource, analysis.getDataSource());
|
||||
Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource());
|
||||
Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuery());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec());
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
|
@ -265,6 +273,7 @@ public class DataSourceAnalysisTest
|
|||
Assert.assertEquals(joinDataSource, analysis.getDataSource());
|
||||
Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource());
|
||||
Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuery());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec());
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
|
@ -321,6 +330,7 @@ public class DataSourceAnalysisTest
|
|||
Assert.assertEquals(joinDataSource, analysis.getDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource());
|
||||
Assert.assertEquals(unionDataSource, analysis.getBaseDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuery());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec());
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
|
@ -354,6 +364,19 @@ public class DataSourceAnalysisTest
|
|||
Assert.assertEquals(queryDataSource, analysis.getDataSource());
|
||||
Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource());
|
||||
Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource());
|
||||
Assert.assertEquals(
|
||||
Optional.of(
|
||||
subquery(
|
||||
join(
|
||||
TABLE_FOO,
|
||||
LOOKUP_LOOKYLOO,
|
||||
"1.",
|
||||
JoinType.INNER
|
||||
)
|
||||
).getQuery()
|
||||
),
|
||||
analysis.getBaseQuery()
|
||||
);
|
||||
Assert.assertEquals(
|
||||
Optional.of(new MultipleIntervalSegmentSpec(MILLENIUM_INTERVALS)),
|
||||
analysis.getBaseQuerySegmentSpec()
|
||||
|
@ -385,6 +408,7 @@ public class DataSourceAnalysisTest
|
|||
Assert.assertEquals(joinDataSource, analysis.getDataSource());
|
||||
Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getBaseDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuery());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec());
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
|
@ -413,6 +437,7 @@ public class DataSourceAnalysisTest
|
|||
Assert.assertEquals(joinDataSource, analysis.getDataSource());
|
||||
Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getBaseDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuery());
|
||||
Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec());
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
|
@ -427,7 +452,10 @@ public class DataSourceAnalysisTest
|
|||
{
|
||||
EqualsVerifier.forClass(DataSourceAnalysis.class)
|
||||
.usingGetClass()
|
||||
.withNonnullFields("dataSource", "baseDataSource", "preJoinableClauses")
|
||||
.withNonnullFields("dataSource")
|
||||
|
||||
// These fields are not necessary, because they're wholly determined by "dataSource"
|
||||
.withIgnoredFields("baseDataSource", "baseQuery", "preJoinableClauses")
|
||||
.verify();
|
||||
}
|
||||
|
||||
|
|
|
@ -28,7 +28,9 @@ import org.apache.druid.query.filter.Filter;
|
|||
import org.apache.druid.query.lookup.LookupExtractor;
|
||||
import org.apache.druid.segment.QueryableIndexSegment;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterAnalyzer;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
import org.apache.druid.segment.join.lookup.LookupJoinable;
|
||||
import org.apache.druid.segment.join.table.IndexedTable;
|
||||
|
@ -193,14 +195,16 @@ public class BaseHashJoinSegmentStorageAdapterTest
|
|||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a fact-to-country join segment without a {@link JoinFilterPreAnalysis}. This means it cannot
|
||||
* have {@link org.apache.druid.segment.StorageAdapter#makeCursors} called on it.
|
||||
*/
|
||||
protected HashJoinSegmentStorageAdapter makeFactToCountrySegment()
|
||||
{
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup();
|
||||
|
||||
return new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)),
|
||||
joinFilterPreAnalysisGroup
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -223,31 +227,19 @@ public class BaseHashJoinSegmentStorageAdapterTest
|
|||
);
|
||||
}
|
||||
|
||||
protected static JoinFilterPreAnalysisGroup makeDefaultConfigPreAnalysisGroup()
|
||||
{
|
||||
return new JoinFilterPreAnalysisGroup(
|
||||
DEFAULT_JOIN_FILTER_REWRITE_CONFIG,
|
||||
true
|
||||
);
|
||||
}
|
||||
|
||||
protected static JoinFilterPreAnalysisGroup makeDefaultConfigPreAnalysisGroup(
|
||||
protected static JoinFilterPreAnalysis makeDefaultConfigPreAnalysis(
|
||||
Filter originalFilter,
|
||||
List<JoinableClause> joinableClauses,
|
||||
VirtualColumns virtualColumns
|
||||
)
|
||||
{
|
||||
JoinFilterPreAnalysisGroup group = new JoinFilterPreAnalysisGroup(
|
||||
DEFAULT_JOIN_FILTER_REWRITE_CONFIG,
|
||||
true
|
||||
return JoinFilterAnalyzer.computeJoinFilterPreAnalysis(
|
||||
new JoinFilterPreAnalysisKey(
|
||||
DEFAULT_JOIN_FILTER_REWRITE_CONFIG,
|
||||
joinableClauses,
|
||||
virtualColumns,
|
||||
originalFilter
|
||||
)
|
||||
);
|
||||
|
||||
group.computeJoinFilterPreAnalysisIfAbsent(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
virtualColumns
|
||||
);
|
||||
|
||||
return group;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.druid.segment.VirtualColumns;
|
|||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.filter.SelectorFilter;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
|
||||
import org.apache.druid.segment.join.lookup.LookupJoinable;
|
||||
import org.apache.druid.segment.join.table.IndexedTableJoinable;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
|
@ -298,7 +298,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
{
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT));
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
null,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -307,7 +307,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
null,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -362,7 +362,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
{
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT));
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
null,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -371,7 +371,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
null,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -424,7 +424,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
public void test_makeCursors_factToCountryInner()
|
||||
{
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.INNER));
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
null,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -433,7 +433,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
null,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -481,7 +481,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
public void test_makeCursors_factToCountryInnerUsingLookup()
|
||||
{
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.INNER));
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
null,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -490,7 +490,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
null,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -540,7 +540,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
// is interpreted as 0 (a.k.a. Australia).
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryOnNumber(JoinType.INNER));
|
||||
Filter filter = new SelectorDimFilter("channel", "#en.wikipedia", null).toFilter();
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -549,7 +549,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -605,7 +605,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
// is interpreted as 0 (a.k.a. Australia).
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryNameUsingNumberLookup(JoinType.INNER));
|
||||
Filter filter = new SelectorDimFilter("channel", "#en.wikipedia", null).toFilter();
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -614,7 +614,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -666,7 +666,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
{
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT));
|
||||
Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter();
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -675,7 +675,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -702,7 +702,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
{
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT));
|
||||
Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter();
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -711,7 +711,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -737,7 +737,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
{
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT));
|
||||
Filter filter = new SelectorDimFilter("channel", null, null).toFilter();
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -746,7 +746,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -775,7 +775,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
{
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.RIGHT));
|
||||
Filter filter = new SelectorDimFilter("channel", null, null).toFilter();
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -784,7 +784,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -812,7 +812,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
{
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL));
|
||||
Filter filter = new SelectorDimFilter("channel", null, null).toFilter();
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -821,7 +821,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -850,7 +850,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
{
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.FULL));
|
||||
Filter filter = new SelectorDimFilter("channel", null, null).toFilter();
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -859,7 +859,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -892,7 +892,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
null
|
||||
).toFilter();
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -901,7 +901,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -934,7 +934,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
null
|
||||
).toFilter();
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -943,7 +943,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -976,7 +976,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber", "10", null)
|
||||
).toFilter();
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -985,7 +985,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1018,7 +1018,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Norway", null)
|
||||
).toFilter();
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1027,7 +1027,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1073,7 +1073,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
ExprMacroTable.nil()
|
||||
).toFilter();
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1082,7 +1082,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1149,7 +1149,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
StringUtils.format("\"%sk\" == countryIsoCode", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX),
|
||||
ExprMacroTable.nil()
|
||||
).toFilter();
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1158,7 +1158,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1208,7 +1208,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
factToRegion(JoinType.LEFT),
|
||||
regionToCountry(JoinType.LEFT)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
null,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1217,7 +1217,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
null,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1282,7 +1282,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
);
|
||||
|
||||
Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter();
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1291,7 +1291,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1345,7 +1345,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
|
||||
Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter();
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1355,7 +1355,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1392,7 +1392,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
|
||||
Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter();
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1401,7 +1401,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1455,7 +1455,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
|
||||
Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter();
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1465,7 +1465,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1511,7 +1511,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
null,
|
||||
joinableClauses,
|
||||
virtualColumns
|
||||
|
@ -1520,7 +1520,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
null,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1572,7 +1572,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
null,
|
||||
joinableClauses,
|
||||
virtualColumns
|
||||
|
@ -1581,7 +1581,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
null,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1625,7 +1625,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
null,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1634,7 +1634,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
null,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1677,7 +1677,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
null,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1686,7 +1686,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
null,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1731,7 +1731,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
);
|
||||
|
||||
Filter filter = new SelectorDimFilter("regionIsoCode", "VA", null).toFilter();
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1740,7 +1740,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1786,7 +1786,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
null,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1796,7 +1796,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
null,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1828,7 +1828,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
null,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1838,7 +1838,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
null,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1870,7 +1870,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
null,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1880,7 +1880,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
null,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1897,7 +1897,8 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
public void test_makeCursors_errorOnNonKeyBasedJoinUsingLookup()
|
||||
{
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Cannot join lookup with condition referring to non-key column: x == \"c1.countryName");
|
||||
expectedException.expectMessage(
|
||||
"Cannot join lookup with condition referring to non-key column: x == \"c1.countryName");
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(
|
||||
new JoinableClause(
|
||||
FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX,
|
||||
|
@ -1911,7 +1912,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
null,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1921,7 +1922,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
null,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1940,7 +1941,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
Filter originalFilter = new SelectorFilter("page", "this matches nothing");
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT));
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1949,7 +1950,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
originalFilter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -1974,7 +1975,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
{
|
||||
Filter originalFilter = new SelectorFilter("page", "this matches nothing");
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT));
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1983,7 +1984,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
originalFilter,
|
||||
Intervals.ETERNITY,
|
||||
|
@ -2007,7 +2008,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
{
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT));
|
||||
Filter filter = new SelectorFilter("page", "this matches nothing");
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -2016,7 +2017,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag
|
|||
new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
).makeCursors(
|
||||
filter,
|
||||
Intervals.ETERNITY,
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.druid.segment.QueryableIndexSegment;
|
|||
import org.apache.druid.segment.ReferenceCountingSegment;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
import org.apache.druid.segment.join.table.IndexedTableJoinable;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
|
@ -50,12 +49,13 @@ import java.util.Optional;
|
|||
|
||||
public class HashJoinSegmentTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private JoinFilterRewriteConfig DEFAULT_JOIN_FILTER_REWRITE_CONFIG = new JoinFilterRewriteConfig(
|
||||
true,
|
||||
true,
|
||||
true,
|
||||
QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE
|
||||
);
|
||||
private static final JoinFilterRewriteConfig DEFAULT_JOIN_FILTER_REWRITE_CONFIG =
|
||||
new JoinFilterRewriteConfig(
|
||||
true,
|
||||
true,
|
||||
true,
|
||||
QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE
|
||||
);
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
@ -134,11 +134,6 @@ public class HashJoinSegmentTest extends InitializedNullHandlingTest
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup(
|
||||
DEFAULT_JOIN_FILTER_REWRITE_CONFIG,
|
||||
true
|
||||
);
|
||||
|
||||
referencedSegment = ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment);
|
||||
SegmentReference testWrapper = new SegmentReference()
|
||||
{
|
||||
|
@ -188,7 +183,7 @@ public class HashJoinSegmentTest extends InitializedNullHandlingTest
|
|||
hashJoinSegment = new HashJoinSegment(
|
||||
testWrapper,
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
null
|
||||
)
|
||||
{
|
||||
@Override
|
||||
|
@ -213,15 +208,10 @@ public class HashJoinSegmentTest extends InitializedNullHandlingTest
|
|||
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of();
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup(
|
||||
DEFAULT_JOIN_FILTER_REWRITE_CONFIG,
|
||||
true
|
||||
);
|
||||
|
||||
final HashJoinSegment ignored = new HashJoinSegment(
|
||||
ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -44,9 +44,9 @@ import org.apache.druid.segment.filter.OrFilter;
|
|||
import org.apache.druid.segment.filter.SelectorFilter;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterAnalyzer;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterSplit;
|
||||
import org.apache.druid.segment.join.filter.JoinableClauses;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
import org.apache.druid.segment.join.lookup.LookupJoinable;
|
||||
import org.apache.druid.segment.join.table.IndexedTableJoinable;
|
||||
|
@ -68,7 +68,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
regionToCountry(JoinType.LEFT)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -77,7 +77,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -117,7 +117,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
null,
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
}
|
||||
|
@ -146,7 +145,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
regionExprToCountry
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -155,7 +154,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -182,7 +181,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter("rtc.countryName", "United States"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
}
|
||||
|
@ -203,7 +201,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
);
|
||||
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -212,7 +210,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -248,7 +246,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter("rtc.countryName", "United States"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -271,7 +268,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
regionToCountry(JoinType.LEFT)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -280,7 +277,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -321,12 +318,10 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns()
|
||||
{
|
||||
|
@ -344,7 +339,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -353,7 +348,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -384,7 +379,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -404,17 +398,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
);
|
||||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
);
|
||||
|
||||
VirtualColumns virtualColumns = VirtualColumns.create(
|
||||
ImmutableList.of(
|
||||
new ExpressionVirtualColumn(
|
||||
|
@ -426,6 +409,18 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
virtualColumns
|
||||
);
|
||||
|
||||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
adapter.makeCursors(
|
||||
originalFilter,
|
||||
|
@ -463,7 +458,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
null,
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -490,7 +484,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
factToRegion(JoinType.LEFT)
|
||||
));
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses.getJoinableClauses(),
|
||||
virtualColumns
|
||||
|
@ -498,7 +492,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses.getJoinableClauses(),
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -524,7 +518,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter("v0", "VIRGINIA"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -595,7 +588,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
regionToCountry(JoinType.LEFT)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -603,7 +596,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -689,7 +682,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -723,7 +715,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -731,7 +723,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -763,7 +755,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter("rtc.countryName", "States United"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
|
||||
|
@ -815,18 +806,19 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
);
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Cannot build hash-join matcher on non-equi-join condition: \"r1.regionIsoCode\" == regionIsoCode && reverse(\"r1.countryIsoCode\") == countryIsoCode");
|
||||
expectedException.expectMessage(
|
||||
"Cannot build hash-join matcher on non-equi-join condition: \"r1.regionIsoCode\" == regionIsoCode && reverse(\"r1.countryIsoCode\") == countryIsoCode");
|
||||
|
||||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
JoinTestHelper.verifyCursors(
|
||||
adapter.makeCursors(
|
||||
|
@ -867,7 +859,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -875,7 +867,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -939,8 +931,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
}
|
||||
|
@ -971,7 +961,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -979,7 +969,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1020,7 +1010,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
expectedVirtualColumn
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(
|
||||
|
@ -1062,7 +1051,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1070,7 +1059,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1111,7 +1100,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
expectedVirtualColumn
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(
|
||||
|
@ -1137,7 +1125,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Germany")
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1145,7 +1133,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1180,7 +1168,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Germany"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -1196,7 +1183,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Germany")
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1204,7 +1191,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1238,7 +1225,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Germany"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -1254,7 +1240,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null)
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1262,7 +1248,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1295,7 +1281,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -1311,7 +1296,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "NO MATCH")
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1319,7 +1304,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1353,7 +1338,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -1369,7 +1353,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null)
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1377,7 +1361,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1409,7 +1393,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -1425,7 +1408,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", "Australia")
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1433,7 +1416,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
// In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman"
|
||||
|
@ -1474,7 +1457,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", "Australia"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -1490,7 +1472,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", "Australia")
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1498,7 +1480,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
// In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman"
|
||||
|
@ -1538,7 +1520,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", "Australia"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -1554,7 +1535,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", null)
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1562,7 +1543,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1594,7 +1575,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -1610,7 +1590,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", null)
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1618,7 +1598,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1649,7 +1629,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -1665,7 +1644,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
)
|
||||
);
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL));
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1673,7 +1652,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1708,7 +1687,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "El Salvador"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -1724,7 +1702,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
)
|
||||
);
|
||||
List<JoinableClause> joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.FULL));
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
filter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1732,7 +1710,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1766,7 +1744,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "El Salvador"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -1782,7 +1759,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null)
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1790,7 +1767,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1823,7 +1800,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -1839,7 +1815,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null)
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1847,7 +1823,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1879,7 +1855,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -1907,7 +1882,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
);
|
||||
Filter originalFilter = new SelectorFilter("r1.regionName", "Fourems Province");
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1915,7 +1890,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -1947,8 +1922,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
}
|
||||
|
@ -1980,7 +1953,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -1988,7 +1961,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -2020,8 +1993,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
}
|
||||
|
@ -2055,7 +2026,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -2063,7 +2034,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
// This query doesn't execute because regionName is not a key column, but we can still check the
|
||||
|
@ -2099,7 +2070,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
null,
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -2114,25 +2084,24 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
));
|
||||
Filter originalFilter = new SelectorFilter("page", "Peremptory norm");
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup(
|
||||
new JoinFilterRewriteConfig(
|
||||
false,
|
||||
true,
|
||||
true,
|
||||
QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE
|
||||
),
|
||||
true
|
||||
);
|
||||
joinFilterPreAnalysisGroup.computeJoinFilterPreAnalysisIfAbsent(
|
||||
originalFilter,
|
||||
joinableClauses.getJoinableClauses(),
|
||||
VirtualColumns.EMPTY
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis(
|
||||
new JoinFilterPreAnalysisKey(
|
||||
new JoinFilterRewriteConfig(
|
||||
false,
|
||||
true,
|
||||
true,
|
||||
QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE
|
||||
),
|
||||
joinableClauses.getJoinableClauses(),
|
||||
VirtualColumns.EMPTY,
|
||||
originalFilter
|
||||
)
|
||||
);
|
||||
|
||||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses.getJoinableClauses(),
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -2159,7 +2128,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter("page", "Peremptory norm"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -2194,25 +2162,25 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
))
|
||||
)
|
||||
);
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup(
|
||||
new JoinFilterRewriteConfig(
|
||||
true,
|
||||
false,
|
||||
true,
|
||||
QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE
|
||||
),
|
||||
true
|
||||
);
|
||||
joinFilterPreAnalysisGroup.computeJoinFilterPreAnalysisIfAbsent(
|
||||
originalFilter,
|
||||
joinableClauses.getJoinableClauses(),
|
||||
VirtualColumns.EMPTY
|
||||
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis(
|
||||
new JoinFilterPreAnalysisKey(
|
||||
new JoinFilterRewriteConfig(
|
||||
true,
|
||||
false,
|
||||
true,
|
||||
QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE
|
||||
),
|
||||
joinableClauses.getJoinableClauses(),
|
||||
VirtualColumns.EMPTY,
|
||||
originalFilter
|
||||
)
|
||||
);
|
||||
|
||||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses.getJoinableClauses(),
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -2267,7 +2235,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -2292,7 +2259,8 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
ExprMacroTable.nil()
|
||||
).toFilter();
|
||||
|
||||
Filter specialSelectorFilter = new SelectorFilter("rtc.countryIsoCode", "CA") {
|
||||
Filter specialSelectorFilter = new SelectorFilter("rtc.countryIsoCode", "CA")
|
||||
{
|
||||
@Override
|
||||
public boolean supportsRequiredColumnRewrite()
|
||||
{
|
||||
|
@ -2330,7 +2298,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
null,
|
||||
null,
|
||||
null
|
||||
))
|
||||
))
|
||||
)
|
||||
),
|
||||
new OrFilter(
|
||||
|
@ -2378,7 +2346,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
regionToCountry(JoinType.LEFT)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -2386,7 +2354,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
String rewrittenCountryIsoCodeColumnName = hasLhsExpressionInJoinCondition
|
||||
|
@ -2452,7 +2420,12 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
null,
|
||||
null
|
||||
)),
|
||||
new InDimFilter(rewrittenCountryIsoCodeColumnName, ImmutableSet.of("CA", "CA2", "CA3"), null, null).toFilter(),
|
||||
new InDimFilter(
|
||||
rewrittenCountryIsoCodeColumnName,
|
||||
ImmutableSet.of("CA", "CA2", "CA3"),
|
||||
null,
|
||||
null
|
||||
).toFilter(),
|
||||
new InDimFilter(rewrittenCountryIsoCodeColumnName, ImmutableSet.of("CA"), null, null).toFilter(),
|
||||
new OrFilter(
|
||||
ImmutableList.of(
|
||||
|
@ -2474,7 +2447,12 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
ImmutableList.of(
|
||||
new SelectorFilter("namespace", "Main"),
|
||||
new SelectorFilter(rewrittenCountryIsoCodeColumnName, "ABCDEF"),
|
||||
new InDimFilter(rewrittenCountryIsoCodeColumnName, ImmutableSet.of("CA"), null, null).toFilter(),
|
||||
new InDimFilter(
|
||||
rewrittenCountryIsoCodeColumnName,
|
||||
ImmutableSet.of("CA"),
|
||||
null,
|
||||
null
|
||||
).toFilter(),
|
||||
new BoundFilter(new BoundDimFilter(
|
||||
rewrittenCountryIsoCodeColumnName,
|
||||
"XYZXYZ",
|
||||
|
@ -2514,7 +2492,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
),
|
||||
expectedVirtualColumns
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -2535,7 +2512,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
regionToCountry(JoinType.LEFT)
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = makeDefaultConfigPreAnalysis(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
|
@ -2544,7 +2521,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -2585,7 +2562,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
originalFilter,
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode();
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -2615,26 +2591,24 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
regionExprToCountry
|
||||
);
|
||||
|
||||
JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup(
|
||||
new JoinFilterRewriteConfig(
|
||||
true,
|
||||
true,
|
||||
true,
|
||||
QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE
|
||||
),
|
||||
false
|
||||
);
|
||||
|
||||
joinFilterPreAnalysisGroup.computeJoinFilterPreAnalysisIfAbsent(
|
||||
originalFilter,
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis(
|
||||
new JoinFilterPreAnalysisKey(
|
||||
new JoinFilterRewriteConfig(
|
||||
true,
|
||||
true,
|
||||
true,
|
||||
QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE
|
||||
),
|
||||
joinableClauses,
|
||||
VirtualColumns.EMPTY,
|
||||
originalFilter
|
||||
)
|
||||
);
|
||||
|
||||
HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter(
|
||||
factSegment.asStorageAdapter(),
|
||||
joinableClauses,
|
||||
joinFilterPreAnalysisGroup
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
|
||||
JoinTestHelper.verifyCursors(
|
||||
|
@ -2661,10 +2635,6 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
new SelectorFilter("rtc.countryName", "United States"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis(
|
||||
originalFilter,
|
||||
VirtualColumns.EMPTY
|
||||
);
|
||||
|
||||
JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis);
|
||||
Assert.assertEquals(expectedFilterSplit, actualFilterSplit);
|
||||
|
@ -2681,11 +2651,11 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes
|
|||
|
||||
|
||||
@Test
|
||||
public void test_JoinFilterPreAnalysisGroupKey_equals()
|
||||
public void test_joinFilterPreAnalysisKey_equals()
|
||||
{
|
||||
EqualsVerifier.forClass(JoinFilterPreAnalysisGroup.JoinFilterPreAnalysisGroupKey.class)
|
||||
EqualsVerifier.forClass(JoinFilterPreAnalysisKey.class)
|
||||
.usingGetClass()
|
||||
.withNonnullFields("filter", "virtualColumns")
|
||||
.withNonnullFields("virtualColumns")
|
||||
.verify();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,19 +23,13 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.JoinDataSource;
|
||||
import org.apache.druid.query.LookupDataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.TableDataSource;
|
||||
import org.apache.druid.query.TestQuery;
|
||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.query.extraction.MapLookupExtractor;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.planning.PreJoinableClause;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
|
@ -47,7 +41,6 @@ import org.junit.Rule;
|
|||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -119,7 +112,6 @@ public class JoinablesTest
|
|||
ImmutableList.of(),
|
||||
NoopJoinableFactory.INSTANCE,
|
||||
new AtomicLong(),
|
||||
DEFAULT_JOIN_FILTER_REWRITE_CONFIG,
|
||||
null
|
||||
);
|
||||
|
||||
|
@ -144,7 +136,6 @@ public class JoinablesTest
|
|||
ImmutableList.of(clause),
|
||||
NoopJoinableFactory.INSTANCE,
|
||||
new AtomicLong(),
|
||||
DEFAULT_JOIN_FILTER_REWRITE_CONFIG,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
@ -188,7 +179,6 @@ public class JoinablesTest
|
|||
}
|
||||
},
|
||||
new AtomicLong(),
|
||||
DEFAULT_JOIN_FILTER_REWRITE_CONFIG,
|
||||
new TestQuery(
|
||||
new TableDataSource("test"),
|
||||
new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))),
|
||||
|
@ -248,83 +238,4 @@ public class JoinablesTest
|
|||
|
||||
Joinables.checkPrefixesForDuplicatesAndShadowing(prefixes);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_gatherAllJoinQueryLevels()
|
||||
{
|
||||
Query query1 = new GroupByQuery.Builder()
|
||||
.addDimension("dim1")
|
||||
.setDataSource(
|
||||
JoinDataSource.create(
|
||||
new NoopDataSource(),
|
||||
new NoopDataSource(),
|
||||
"111",
|
||||
"1",
|
||||
JoinType.LEFT,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.setInterval("1999/2000")
|
||||
.setGranularity(Granularities.YEAR)
|
||||
.build();
|
||||
|
||||
Query query2 = new GroupByQuery.Builder()
|
||||
.addDimension("dim2")
|
||||
.setDataSource(
|
||||
JoinDataSource.create(
|
||||
new NoopDataSource(),
|
||||
new NoopDataSource(),
|
||||
"222",
|
||||
"1",
|
||||
JoinType.LEFT,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.setInterval("1999/2000")
|
||||
.setGranularity(Granularities.YEAR)
|
||||
.build();
|
||||
|
||||
QueryDataSource queryDataSource1 = new QueryDataSource(query1);
|
||||
QueryDataSource queryDataSource2 = new QueryDataSource(query2);
|
||||
|
||||
Query query3 = new GroupByQuery.Builder()
|
||||
.addDimension("dim2")
|
||||
.setDataSource(
|
||||
JoinDataSource.create(
|
||||
JoinDataSource.create(
|
||||
queryDataSource1,
|
||||
new NoopDataSource(),
|
||||
"444",
|
||||
"4",
|
||||
JoinType.LEFT,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
queryDataSource2,
|
||||
"333",
|
||||
"3",
|
||||
JoinType.LEFT,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.setInterval("1999/2000")
|
||||
.setGranularity(Granularities.YEAR)
|
||||
.build();
|
||||
|
||||
Query queryOuter = new GroupByQuery.Builder()
|
||||
.addDimension("dim")
|
||||
.setDataSource(
|
||||
new QueryDataSource(query3)
|
||||
)
|
||||
.setInterval("1999/2000")
|
||||
.setGranularity(Granularities.YEAR)
|
||||
.build();
|
||||
|
||||
List<Query> joinQueryLevels = new ArrayList<>();
|
||||
Joinables.gatherAllJoinQueryLevels(queryOuter, joinQueryLevels);
|
||||
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(query3, query1, query2),
|
||||
joinQueryLevels
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.druid.query.FinalizeResultsQueryRunner;
|
|||
import org.apache.druid.query.MetricsEmittingQueryRunner;
|
||||
import org.apache.druid.query.NoopQueryRunner;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.QueryMetrics;
|
||||
import org.apache.druid.query.QueryRunner;
|
||||
|
@ -59,7 +58,6 @@ import org.apache.druid.query.spec.SpecificSegmentSpec;
|
|||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.join.JoinableFactory;
|
||||
import org.apache.druid.segment.join.Joinables;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
import org.apache.druid.segment.realtime.FireHydrant;
|
||||
import org.apache.druid.segment.realtime.plumber.Sink;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
|
@ -170,20 +168,12 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker
|
|||
throw new ISE("Cannot handle subquery: %s", analysis.getDataSource());
|
||||
}
|
||||
|
||||
final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig(
|
||||
QueryContexts.getEnableJoinFilterPushDown(query),
|
||||
QueryContexts.getEnableJoinFilterRewrite(query),
|
||||
QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query),
|
||||
QueryContexts.getJoinFilterRewriteMaxSize(query)
|
||||
);
|
||||
|
||||
// segmentMapFn maps each base Segment into a joined Segment if necessary.
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = Joinables.createSegmentMapFn(
|
||||
analysis.getPreJoinableClauses(),
|
||||
joinableFactory,
|
||||
cpuTimeAccumulator,
|
||||
joinFilterRewriteConfig,
|
||||
query
|
||||
analysis.getBaseQuery().orElse(query)
|
||||
);
|
||||
|
||||
Iterable<QueryRunner<T>> perSegmentRunners = Iterables.transform(
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.server;
|
||||
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.client.SegmentServerSelector;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
|
@ -28,8 +27,6 @@ import org.apache.druid.java.util.common.guava.FunctionalIterable;
|
|||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import org.apache.druid.query.FluentQueryRunnerBuilder;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.QueryPlus;
|
||||
import org.apache.druid.query.QueryRunner;
|
||||
import org.apache.druid.query.QueryRunnerFactory;
|
||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||
|
@ -37,16 +34,12 @@ import org.apache.druid.query.QuerySegmentWalker;
|
|||
import org.apache.druid.query.SegmentDescriptor;
|
||||
import org.apache.druid.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.segment.ReferenceCountingSegment;
|
||||
import org.apache.druid.segment.Segment;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.SegmentWrangler;
|
||||
import org.apache.druid.segment.join.JoinableFactory;
|
||||
import org.apache.druid.segment.join.Joinables;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
@ -98,24 +91,17 @@ public class LocalQuerySegmentWalker implements QuerySegmentWalker
|
|||
final Iterable<ReferenceCountingSegment> segments =
|
||||
FunctionalIterable.create(segmentWrangler.getSegmentsForIntervals(analysis.getBaseDataSource(), intervals))
|
||||
.transform(ReferenceCountingSegment::wrapRootGenerationSegment);
|
||||
final Query<T> prioritizedAndLaned = prioritizeAndLaneQuery(query, segments);
|
||||
|
||||
final AtomicLong cpuAccumulator = new AtomicLong(0L);
|
||||
final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig(
|
||||
QueryContexts.getEnableJoinFilterPushDown(prioritizedAndLaned),
|
||||
QueryContexts.getEnableJoinFilterRewrite(prioritizedAndLaned),
|
||||
QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(prioritizedAndLaned),
|
||||
QueryContexts.getJoinFilterRewriteMaxSize(prioritizedAndLaned)
|
||||
);
|
||||
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = Joinables.createSegmentMapFn(
|
||||
analysis.getPreJoinableClauses(),
|
||||
joinableFactory,
|
||||
cpuAccumulator,
|
||||
joinFilterRewriteConfig,
|
||||
query
|
||||
analysis.getBaseQuery().orElse(query)
|
||||
);
|
||||
|
||||
final QueryRunnerFactory<T, Query<T>> queryRunnerFactory = conglomerate.findFactory(prioritizedAndLaned);
|
||||
final QueryRunnerFactory<T, Query<T>> queryRunnerFactory = conglomerate.findFactory(query);
|
||||
final QueryRunner<T> baseRunner = queryRunnerFactory.mergeRunners(
|
||||
Execs.directExecutor(),
|
||||
() -> StreamSupport.stream(segments.spliterator(), false)
|
||||
|
@ -139,13 +125,4 @@ public class LocalQuerySegmentWalker implements QuerySegmentWalker
|
|||
// SegmentWranglers only work based on intervals and cannot run with specific segments.
|
||||
throw new ISE("Cannot run with specific segments");
|
||||
}
|
||||
|
||||
private <T> Query<T> prioritizeAndLaneQuery(Query<T> query, Iterable<? extends Segment> segments)
|
||||
{
|
||||
Set<SegmentServerSelector> segmentServerSelectors = new HashSet<>();
|
||||
for (Segment s : segments) {
|
||||
segmentServerSelectors.add(new SegmentServerSelector(s.getId().toDescriptor()));
|
||||
}
|
||||
return scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(query), segmentServerSelectors);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,7 +39,6 @@ import org.apache.druid.query.NoopQueryRunner;
|
|||
import org.apache.druid.query.PerSegmentOptimizingQueryRunner;
|
||||
import org.apache.druid.query.PerSegmentQueryOptimizationContext;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.QueryMetrics;
|
||||
import org.apache.druid.query.QueryRunner;
|
||||
|
@ -57,7 +56,6 @@ import org.apache.druid.segment.ReferenceCountingSegment;
|
|||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.join.JoinableFactory;
|
||||
import org.apache.druid.segment.join.Joinables;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
import org.apache.druid.server.SegmentManager;
|
||||
import org.apache.druid.server.SetAndVerifyContextQueryRunner;
|
||||
import org.apache.druid.server.initialization.ServerConfig;
|
||||
|
@ -193,20 +191,12 @@ public class ServerManager implements QuerySegmentWalker
|
|||
return new NoopQueryRunner<>();
|
||||
}
|
||||
|
||||
final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig(
|
||||
QueryContexts.getEnableJoinFilterPushDown(query),
|
||||
QueryContexts.getEnableJoinFilterRewrite(query),
|
||||
QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query),
|
||||
QueryContexts.getJoinFilterRewriteMaxSize(query)
|
||||
);
|
||||
|
||||
// segmentMapFn maps each base Segment into a joined Segment if necessary.
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = Joinables.createSegmentMapFn(
|
||||
analysis.getPreJoinableClauses(),
|
||||
joinableFactory,
|
||||
cpuTimeAccumulator,
|
||||
joinFilterRewriteConfig,
|
||||
query
|
||||
analysis.getBaseQuery().orElse(query)
|
||||
);
|
||||
|
||||
FunctionalIterable<QueryRunner<T>> queryRunners = FunctionalIterable
|
||||
|
|
|
@ -275,7 +275,7 @@ public class ClientQuerySegmentWalkerTest
|
|||
);
|
||||
|
||||
Assert.assertEquals(1, scheduler.getTotalRun().get());
|
||||
Assert.assertEquals(1, scheduler.getTotalPrioritizedAndLaned().get());
|
||||
Assert.assertEquals(0, scheduler.getTotalPrioritizedAndLaned().get());
|
||||
Assert.assertEquals(1, scheduler.getTotalAcquired().get());
|
||||
Assert.assertEquals(1, scheduler.getTotalReleased().get());
|
||||
}
|
||||
|
@ -318,7 +318,7 @@ public class ClientQuerySegmentWalkerTest
|
|||
// note: this should really be 1, but in the interim queries that are composed of multiple queries count each
|
||||
// invocation of either the cluster or local walker in ClientQuerySegmentWalker
|
||||
Assert.assertEquals(2, scheduler.getTotalRun().get());
|
||||
Assert.assertEquals(2, scheduler.getTotalPrioritizedAndLaned().get());
|
||||
Assert.assertEquals(1, scheduler.getTotalPrioritizedAndLaned().get());
|
||||
Assert.assertEquals(2, scheduler.getTotalAcquired().get());
|
||||
Assert.assertEquals(2, scheduler.getTotalReleased().get());
|
||||
}
|
||||
|
@ -506,7 +506,7 @@ public class ClientQuerySegmentWalkerTest
|
|||
);
|
||||
|
||||
Assert.assertEquals(2, scheduler.getTotalRun().get());
|
||||
Assert.assertEquals(2, scheduler.getTotalPrioritizedAndLaned().get());
|
||||
Assert.assertEquals(1, scheduler.getTotalPrioritizedAndLaned().get());
|
||||
Assert.assertEquals(2, scheduler.getTotalAcquired().get());
|
||||
Assert.assertEquals(2, scheduler.getTotalReleased().get());
|
||||
}
|
||||
|
@ -561,7 +561,7 @@ public class ClientQuerySegmentWalkerTest
|
|||
);
|
||||
|
||||
Assert.assertEquals(2, scheduler.getTotalRun().get());
|
||||
Assert.assertEquals(2, scheduler.getTotalPrioritizedAndLaned().get());
|
||||
Assert.assertEquals(1, scheduler.getTotalPrioritizedAndLaned().get());
|
||||
Assert.assertEquals(2, scheduler.getTotalAcquired().get());
|
||||
Assert.assertEquals(2, scheduler.getTotalReleased().get());
|
||||
}
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.druid.query.FinalizeResultsQueryRunner;
|
|||
import org.apache.druid.query.NoopQueryRunner;
|
||||
import org.apache.druid.query.Queries;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.QueryRunner;
|
||||
import org.apache.druid.query.QueryRunnerFactory;
|
||||
|
@ -48,7 +47,6 @@ import org.apache.druid.segment.Segment;
|
|||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.join.JoinableFactory;
|
||||
import org.apache.druid.segment.join.Joinables;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
import org.apache.druid.timeline.TimelineObjectHolder;
|
||||
import org.apache.druid.timeline.VersionedIntervalTimeline;
|
||||
import org.apache.druid.timeline.partition.PartitionChunk;
|
||||
|
@ -140,19 +138,12 @@ public class TestClusterQuerySegmentWalker implements QuerySegmentWalker
|
|||
&& !toolChest.canPerformSubquery(((QueryDataSource) analysis.getDataSource()).getQuery())) {
|
||||
throw new ISE("Cannot handle subquery: %s", analysis.getDataSource());
|
||||
}
|
||||
final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig(
|
||||
QueryContexts.getEnableJoinFilterPushDown(query),
|
||||
QueryContexts.getEnableJoinFilterRewrite(query),
|
||||
QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query),
|
||||
QueryContexts.getJoinFilterRewriteMaxSize(query)
|
||||
);
|
||||
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = Joinables.createSegmentMapFn(
|
||||
analysis.getPreJoinableClauses(),
|
||||
joinableFactory,
|
||||
new AtomicLong(),
|
||||
joinFilterRewriteConfig,
|
||||
query
|
||||
analysis.getBaseQuery().orElse(query)
|
||||
);
|
||||
|
||||
final QueryRunner<T> baseRunner = new FinalizeResultsQueryRunner<>(
|
||||
|
@ -221,7 +212,8 @@ public class TestClusterQuerySegmentWalker implements QuerySegmentWalker
|
|||
.transform(
|
||||
segment ->
|
||||
new SpecificSegmentQueryRunner<>(
|
||||
factory.createRunner(segmentMapFn.apply(ReferenceCountingSegment.wrapRootGenerationSegment(segment.getSegment()))),
|
||||
factory.createRunner(segmentMapFn.apply(ReferenceCountingSegment.wrapRootGenerationSegment(
|
||||
segment.getSegment()))),
|
||||
new SpecificSegmentSpec(segment.getDescriptor())
|
||||
)
|
||||
)
|
||||
|
|
Loading…
Reference in New Issue