Skip to content

Commit

Permalink
implement node level searchcontext registry
Browse files Browse the repository at this point in the history
waiting for job-scoped shard sequence ids
  • Loading branch information
msbt committed Feb 24, 2015
1 parent ff9b8e5 commit 67d27ee
Show file tree
Hide file tree
Showing 13 changed files with 298 additions and 120 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import io.crate.breaker.RamAccountingContext;
import io.crate.executor.QueryResult;
import io.crate.executor.JobTask;
import io.crate.executor.QueryResult;
import io.crate.executor.TaskResult;
import io.crate.operation.collect.CollectOperation;
import io.crate.planner.node.dql.CollectNode;
Expand All @@ -41,7 +41,7 @@


/**
* A collect task which returns one future and runs a collectOperation locally and synchronous
* A collect task which returns one future and runs a collectOperation locally
*/
public class LocalCollectTask extends JobTask {

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,118 @@
/*
* Licensed to CRATE Technology GmbH ("Crate") under one or more contributor
* license agreements. See the NOTICE file distributed with this work for
* additional information regarding copyright ownership. Crate 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.
*
* However, if you have executed another commercial license agreement
* with Crate these terms will supersede the license and you may use the
* software solely pursuant to the terms of the relevant commercial agreement.
*/

package io.crate.executor.transport;

import com.carrotsearch.hppc.IntObjectMap;
import com.carrotsearch.hppc.IntObjectOpenHashMap;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.google.common.base.Throwables;
import com.google.common.cache.*;
import com.google.common.util.concurrent.UncheckedExecutionException;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.search.internal.SearchContext;

import javax.annotation.Nonnull;
import java.util.UUID;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantLock;

public class CollectContextService implements Releasable {

public static final String EXPIRATION_SETTING = "collect.context_keep_alive";
public static final TimeValue EXPIRATION_DEFAULT = new TimeValue(5, TimeUnit.MINUTES);

private final ReentrantLock lock;
private final LoadingCache<UUID, IntObjectMap<SearchContext>> contexts;

@Inject
public CollectContextService(Settings settings) {
long expirationMillis = settings.getAsTime(EXPIRATION_SETTING, EXPIRATION_DEFAULT).millis();
if (expirationMillis == 0) {
Loggers.getLogger(getClass()).info("invalid {}. using default of {}", EXPIRATION_SETTING, EXPIRATION_DEFAULT);
expirationMillis = EXPIRATION_DEFAULT.millis();
}
this.contexts = CacheBuilder.newBuilder().expireAfterAccess(
expirationMillis,
TimeUnit.MILLISECONDS
).removalListener(new RemovalListener<UUID, IntObjectMap<SearchContext>>() {
@Override
public void onRemoval(@Nonnull RemovalNotification<UUID, IntObjectMap<SearchContext>> notification) {
for (ObjectCursor<SearchContext> cursor : notification.getValue().values()) {
Releasables.close(cursor.value);
}
}
}).build(new CacheLoader<UUID, IntObjectMap<SearchContext>>() {
@Override
public IntObjectMap<SearchContext> load(@Nonnull UUID key) throws Exception {
return new IntObjectOpenHashMap<>();
}
});
this.lock = new ReentrantLock();
}


@Nullable
public SearchContext getContext(UUID jobId, int searchContextId) {
final IntObjectMap<SearchContext> searchContexts;
try {
searchContexts = contexts.get(jobId);
} catch (ExecutionException|UncheckedExecutionException e) {
throw Throwables.propagate(e);
}
return searchContexts.get(searchContextId);
}

public SearchContext getOrCreateContext(UUID jobId,
int searchContextId,
Callable<SearchContext> createSearchContextCallable) {
try {
final IntObjectMap<SearchContext> searchContexts = contexts.get(jobId);
lock.lock();
try {
SearchContext searchContext = searchContexts.get(searchContextId);
if (searchContext == null) {
searchContext = createSearchContextCallable.call();
searchContexts.put(searchContextId, searchContext);
}
return searchContext;
} finally {
lock.unlock();
}
} catch (Exception e) {
throw Throwables.propagate(e);
}
}

@Override
public void close() throws ElasticsearchException {
this.contexts.invalidateAll();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -104,11 +104,11 @@ private void nodeOperation(final NodeCollectRequest request,
operationId = UUID.randomUUID();
statsTables.operationStarted(operationId, node.jobId().get(), node.id());
} else {
operationId = null;
collectResponse.onFailure(new IllegalArgumentException("no jobId given for CollectOperation"));
return;
}
String ramAccountingContextId = String.format("%s: %s", node.id(), operationId);
final RamAccountingContext ramAccountingContext =
new RamAccountingContext(ramAccountingContextId, circuitBreaker);
final RamAccountingContext ramAccountingContext = new RamAccountingContext(ramAccountingContextId, circuitBreaker);

try {
if (node.hasDownstreams()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,8 @@ public class TransportExecutor implements Executor, TaskExecutor {

private final QueryThenFetchOperation queryThenFetchOperation;

private final CollectContextService collectContextService;

@Inject
public TransportExecutor(Settings settings,
TransportActionProvider transportActionProvider,
Expand All @@ -93,7 +95,8 @@ public TransportExecutor(Settings settings,
StatsTables statsTables,
ClusterService clusterService,
CrateCircuitBreakerService breakerService,
QueryThenFetchOperation queryThenFetchOperation) {
QueryThenFetchOperation queryThenFetchOperation,
CollectContextService collectContextService) {
this.settings = settings;
this.transportActionProvider = transportActionProvider;
this.handlerSideDataCollectOperation = handlerSideDataCollectOperation;
Expand All @@ -103,6 +106,7 @@ public TransportExecutor(Settings settings,
this.statsTables = statsTables;
this.clusterService = clusterService;
this.queryThenFetchOperation = queryThenFetchOperation;
this.collectContextService = collectContextService;
this.nodeVisitor = new NodeVisitor();
this.planVisitor = new TaskCollectingVisitor();
this.circuitBreaker = breakerService.getBreaker(CrateCircuitBreakerService.QUERY_BREAKER);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,14 @@

package io.crate.operation.collect;

import io.crate.executor.transport.CollectContextService;
import org.elasticsearch.common.inject.AbstractModule;

public class CollectOperationModule extends AbstractModule {
@Override
protected void configure() {
bind(CollectContextService.class).asEagerSingleton();

bind(MapSideDataCollectOperation.class).asEagerSingleton();
bind(HandlerSideDataCollectOperation.class).asEagerSingleton();
bind(InformationSchemaCollectService.class).asEagerSingleton();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -333,7 +333,8 @@ public DiscoveryNode apply(@Nullable String input) {
}

@Override
protected ListenableFuture<Object[][]> handleShardCollect(CollectNode collectNode, RamAccountingContext ramAccountingContext) {
protected ListenableFuture<Object[][]> handleShardCollect(CollectNode collectNode,
RamAccountingContext ramAccountingContext) {
assert collectNode.hasDownstreams() : "no downstreams";
return super.handleShardCollect(collectNode, ramAccountingContext);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import io.crate.analyze.WhereClause;
import io.crate.breaker.CrateCircuitBreakerService;
import io.crate.breaker.RamAccountingContext;
import io.crate.executor.transport.CollectContextService;
import io.crate.lucene.LuceneQueryBuilder;
import io.crate.metadata.Functions;
import io.crate.operation.Input;
Expand Down Expand Up @@ -56,6 +57,9 @@
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.UUID;
import java.util.concurrent.Callable;

/**
* collect documents from ES shard, a lucene index
Expand Down Expand Up @@ -98,54 +102,69 @@ public void required(boolean required) {
}
}

private final SearchContext searchContext;
private Projector downstream;
private final List<Input<?>> topLevelInputs;
private final List<LuceneCollectorExpression<?>> collectorExpressions;
private final SearchContext searchContext;

public LuceneDocCollector(ThreadPool threadPool,
public LuceneDocCollector(final UUID jobId,
final ThreadPool threadPool,
ClusterService clusterService,
CollectContextService collectorContextService,
ShardId shardId,
IndexService indexService,
ScriptService scriptService,
CacheRecycler cacheRecycler,
PageCacheRecycler pageCacheRecycler,
BigArrays bigArrays,
final IndexService indexService,
final ScriptService scriptService,
final CacheRecycler cacheRecycler,
final PageCacheRecycler pageCacheRecycler,
final BigArrays bigArrays,
List<Input<?>> inputs,
List<LuceneCollectorExpression<?>> collectorExpressions,
Functions functions,
WhereClause whereClause,
final Functions functions,
final WhereClause whereClause,
Projector downStreamProjector) throws Exception {
downstream(downStreamProjector);
SearchShardTarget searchShardTarget = new SearchShardTarget(
final SearchShardTarget searchShardTarget = new SearchShardTarget(
clusterService.localNode().id(), shardId.getIndex(), shardId.id());
this.topLevelInputs = inputs;
this.collectorExpressions = collectorExpressions;
this.fieldsVisitor = new CollectorFieldsVisitor(collectorExpressions.size());

ShardSearchLocalRequest searchRequest = new ShardSearchLocalRequest(
new String[] { Constants.DEFAULT_MAPPING_TYPE },
System.currentTimeMillis()
final IndexShard indexShard = indexService.shardSafe(shardId.id());
final int searchContextId = Objects.hash(jobId, searchShardTarget);
this.searchContext = collectorContextService.getOrCreateContext(
jobId,
searchContextId,
new Callable<SearchContext>() {
@Override
public SearchContext call() throws Exception {
ShardSearchLocalRequest searchRequest = new ShardSearchLocalRequest(
new String[] { Constants.DEFAULT_MAPPING_TYPE },
System.currentTimeMillis()
);
SearchContext localContext = new DefaultSearchContext(
searchContextId,
searchRequest,
searchShardTarget,
EngineSearcher.getSearcherWithRetry(indexShard, null), // TODO: use same searcher/reader for same jobId and searchContextId
indexService,
indexShard,
scriptService,
cacheRecycler,
pageCacheRecycler,
bigArrays,
threadPool.estimatedTimeInMillisCounter()
);
LuceneQueryBuilder builder = new LuceneQueryBuilder(functions, localContext, indexService.cache());
LuceneQueryBuilder.Context ctx = builder.convert(whereClause);
localContext.parsedQuery(new ParsedQuery(ctx.query(), ImmutableMap.<String, Filter>of()));
Float minScore = ctx.minScore();
if (minScore != null) {
localContext.minimumScore(minScore);
}
return localContext;
}
}
);
IndexShard indexShard = indexService.shardSafe(shardId.id());
searchContext = new DefaultSearchContext(0, searchRequest,
searchShardTarget,
EngineSearcher.getSearcherWithRetry(indexShard, null),
indexService,
indexShard,
scriptService,
cacheRecycler,
pageCacheRecycler,
bigArrays,
threadPool.estimatedTimeInMillisCounter()
);
LuceneQueryBuilder builder = new LuceneQueryBuilder(functions, searchContext, indexService.cache());
LuceneQueryBuilder.Context ctx = builder.convert(whereClause);
searchContext.parsedQuery(new ParsedQuery(ctx.query(), ImmutableMap.<String, Filter>of()));
Float minScore = ctx.minScore();
if (minScore != null) {
searchContext.minimumScore(minScore);
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,7 @@ public MapSideDataCollectOperation(ClusterService clusterService,
@Override
public ListenableFuture<Object[][]> collect(CollectNode collectNode, RamAccountingContext ramAccountingContext) {
assert collectNode.isRouted(); // not routed collect is not handled here
assert collectNode.jobId().isPresent() : "no jobId present for collect operation";
String localNodeId = clusterService.localNode().id();
if (collectNode.executionNodes().contains(localNodeId)) {
if (!collectNode.routing().containsShards(localNodeId)) {
Expand Down

0 comments on commit 67d27ee

Please sign in to comment.