mirror of https://github.com/apache/druid.git
Addressing review comments part 1
This commit is contained in:
parent
557081a11e
commit
1b75daf811
|
@ -157,11 +157,12 @@ public abstract class BaseLeafFrameProcessor implements FrameProcessor<Long>
|
||||||
final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs);
|
final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs);
|
||||||
if (retVal) {
|
if (retVal) {
|
||||||
InputNumberDataSource inputNumberDataSource = (InputNumberDataSource) query.getDataSource();
|
InputNumberDataSource inputNumberDataSource = (InputNumberDataSource) query.getDataSource();
|
||||||
// The InputNumberData source was going through the broadcastJoinHelper which
|
// The InputNumberDataSource requires a BroadcastJoinHelper to be able to create its
|
||||||
// was using the JoinableFactoryWrapper to create segment map function.
|
// segment map function. It would be a lot better if the InputNumberDataSource actually
|
||||||
// After refactoring, the segment map function creation is moved to data source
|
// had a way to get that injected into it on its own, but the relationship between these objects
|
||||||
// Hence for InputNumberDataSource we are setting the broadcast join helper for the data source
|
// was figured out during a refactor and using a setter here seemed like the least-bad way to
|
||||||
// and moving the segment map function creation there
|
// make progress on the refactor without breaking functionality. Hopefully, some future
|
||||||
|
// developer will move this away from a setter.
|
||||||
inputNumberDataSource.setBroadcastJoinHelper(broadcastJoinHelper);
|
inputNumberDataSource.setBroadcastJoinHelper(broadcastJoinHelper);
|
||||||
segmentMapFn = inputNumberDataSource.createSegmentMapFunction(query, new AtomicLong());
|
segmentMapFn = inputNumberDataSource.createSegmentMapFunction(query, new AtomicLong());
|
||||||
}
|
}
|
||||||
|
|
|
@ -573,16 +573,6 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
||||||
)
|
)
|
||||||
).registerSubtypes(ExternalDataSource.class));
|
).registerSubtypes(ExternalDataSource.class));
|
||||||
DruidSecondaryModule.setupJackson(injector, mapper);
|
DruidSecondaryModule.setupJackson(injector, mapper);
|
||||||
/*
|
|
||||||
mapper.setInjectableValues(
|
|
||||||
new InjectableValues.Std()
|
|
||||||
.addValue(ObjectMapper.class, mapper)
|
|
||||||
.addValue(Injector.class, injector)
|
|
||||||
.addValue(DataSegment.PruneSpecsHolder.class, DataSegment.PruneSpecsHolder.DEFAULT)
|
|
||||||
.addValue(LocalDataSegmentPuller.class, new LocalDataSegmentPuller())
|
|
||||||
.addValue(ExprMacroTable.class, CalciteTests.createExprMacroTable())
|
|
||||||
);
|
|
||||||
*/
|
|
||||||
|
|
||||||
mapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local"));
|
mapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local"));
|
||||||
|
|
||||||
|
|
|
@ -156,7 +156,7 @@ public class JoinDataSource implements DataSource
|
||||||
final JoinConditionAnalysis conditionAnalysis,
|
final JoinConditionAnalysis conditionAnalysis,
|
||||||
final JoinType joinType,
|
final JoinType joinType,
|
||||||
final DimFilter leftFilter,
|
final DimFilter leftFilter,
|
||||||
@Nullable @JacksonInject final JoinableFactoryWrapper joinableFactoryWrapper
|
@Nullable final JoinableFactoryWrapper joinableFactoryWrapper
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return new JoinDataSource(
|
return new JoinDataSource(
|
||||||
|
|
|
@ -1,3 +1,22 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
package org.apache.druid.segment.join;
|
||||||
|
|
||||||
import org.apache.druid.query.DataSource;
|
import org.apache.druid.query.DataSource;
|
||||||
|
@ -24,4 +43,4 @@ public class NoopJoinableFactory implements JoinableFactory
|
||||||
{
|
{
|
||||||
return Optional.empty();
|
return Optional.empty();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,46 +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;
|
|
||||||
|
|
||||||
import org.apache.druid.query.DataSource;
|
|
||||||
|
|
||||||
import java.util.Optional;
|
|
||||||
|
|
||||||
public class NoopJoinableFactory implements JoinableFactory
|
|
||||||
{
|
|
||||||
public static final NoopJoinableFactory INSTANCE = new NoopJoinableFactory();
|
|
||||||
|
|
||||||
protected NoopJoinableFactory()
|
|
||||||
{
|
|
||||||
// Singleton.
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean isDirectlyJoinable(DataSource dataSource)
|
|
||||||
{
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Optional<Joinable> build(DataSource dataSource, JoinConditionAnalysis condition)
|
|
||||||
{
|
|
||||||
return Optional.empty();
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -108,9 +108,9 @@ public class CacheUtil
|
||||||
ServerType serverType
|
ServerType serverType
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return isQueryCacheable(query, cacheStrategy, cacheConfig, serverType)
|
return cacheConfig.isUseCache()
|
||||||
&& QueryContexts.isUseCache(query)
|
&& QueryContexts.isUseCache(query)
|
||||||
&& cacheConfig.isUseCache();
|
&& isQueryCacheable(query, cacheStrategy, cacheConfig, serverType);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -93,11 +93,12 @@ public class LocalQuerySegmentWalker implements QuerySegmentWalker
|
||||||
|
|
||||||
final AtomicLong cpuAccumulator = new AtomicLong(0L);
|
final AtomicLong cpuAccumulator = new AtomicLong(0L);
|
||||||
|
|
||||||
Function<SegmentReference, SegmentReference> segmentMapFn = analysis.getDataSource()
|
Function<SegmentReference, SegmentReference> segmentMapFn = analysis
|
||||||
.createSegmentMapFunction(
|
.getDataSource()
|
||||||
query,
|
.createSegmentMapFunction(
|
||||||
cpuAccumulator
|
query,
|
||||||
);
|
cpuAccumulator
|
||||||
|
);
|
||||||
|
|
||||||
|
|
||||||
final QueryRunnerFactory<T, Query<T>> queryRunnerFactory = conglomerate.findFactory(query);
|
final QueryRunnerFactory<T, Query<T>> queryRunnerFactory = conglomerate.findFactory(query);
|
||||||
|
|
|
@ -26,7 +26,6 @@ import com.google.inject.Key;
|
||||||
import com.google.inject.Module;
|
import com.google.inject.Module;
|
||||||
import com.google.inject.TypeLiteral;
|
import com.google.inject.TypeLiteral;
|
||||||
import com.google.inject.name.Names;
|
import com.google.inject.name.Names;
|
||||||
import com.google.inject.util.Providers;
|
|
||||||
import org.apache.druid.curator.discovery.DiscoveryModule;
|
import org.apache.druid.curator.discovery.DiscoveryModule;
|
||||||
import org.apache.druid.discovery.NodeRole;
|
import org.apache.druid.discovery.NodeRole;
|
||||||
import org.apache.druid.guice.Jerseys;
|
import org.apache.druid.guice.Jerseys;
|
||||||
|
@ -43,7 +42,6 @@ import org.apache.druid.java.util.common.logger.Logger;
|
||||||
import org.apache.druid.query.QuerySegmentWalker;
|
import org.apache.druid.query.QuerySegmentWalker;
|
||||||
import org.apache.druid.query.lookup.LookupSerdeModule;
|
import org.apache.druid.query.lookup.LookupSerdeModule;
|
||||||
import org.apache.druid.segment.join.JoinableFactory;
|
import org.apache.druid.segment.join.JoinableFactory;
|
||||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
|
||||||
import org.apache.druid.segment.join.NoopJoinableFactory;
|
import org.apache.druid.segment.join.NoopJoinableFactory;
|
||||||
import org.apache.druid.server.AsyncQueryForwardingServlet;
|
import org.apache.druid.server.AsyncQueryForwardingServlet;
|
||||||
import org.apache.druid.server.NoopQuerySegmentWalker;
|
import org.apache.druid.server.NoopQuerySegmentWalker;
|
||||||
|
|
Loading…
Reference in New Issue