Skip to content

Commit

Permalink
added support for kill statement up to task
Browse files Browse the repository at this point in the history
which throws an UnsupportedOperationException
  • Loading branch information
msbt committed May 7, 2015
1 parent 326e843 commit 84abe9a
Show file tree
Hide file tree
Showing 16 changed files with 319 additions and 9 deletions.
13 changes: 9 additions & 4 deletions sql-parser/src/main/java/io/crate/sql/parser/Statement.g
Original file line number Diff line number Diff line change
Expand Up @@ -205,6 +205,7 @@ statement
| refreshStmt
| setStmt
| resetStmt
| killStmt
;

query
Expand Down Expand Up @@ -1017,16 +1018,19 @@ settingsType
| PERSISTENT
;

killStmt
: KILL ALL -> ^(KILL ALL)
;

nonReserved
: ALIAS | ANALYZER | BERNOULLI | BLOB | CATALOGS | CHAR_FILTERS | CLUSTERED
| COLUMNS | COPY | CURRENT | DATE | DAY | DISTRIBUTED | DYNAMIC | EXPLAIN
| COLUMNS | COPY | CURRENT | DATE | DAY | DISTRIBUTED | DUPLICATE | DYNAMIC | EXPLAIN
| EXTENDS | FOLLOWING | FORMAT | FULLTEXT | FUNCTIONS | GEO_POINT | GLOBAL
| GRAPHVIZ | HOUR | IGNORED | INTERVAL | LOGICAL | MATERIALIZED | MINUTE
| GRAPHVIZ | HOUR | IGNORED | INTERVAL | KEY | KILL | LOGICAL | MATERIALIZED | MINUTE
| MONTH | OFF | OVER | PARTITION | PARTITIONED | PARTITIONS | PLAIN
| PRECEDING | RANGE | REFRESH | ROW | ROWS | SCHEMAS | SECOND
| SHARDS | SHOW | STRICT | SYSTEM | TABLES | TABLESAMPLE | TEXT | TIME
| TIMESTAMP | TO | TOKENIZER | TOKEN_FILTERS | TYPE | VIEW | YEAR
| DUPLICATE | KEY | VALUES
| TIMESTAMP | TO | TOKENIZER | TOKEN_FILTERS | TYPE | VALUES | VIEW | YEAR
;

SELECT: 'SELECT';
Expand Down Expand Up @@ -1110,6 +1114,7 @@ CREATE: 'CREATE';
BLOB: 'BLOB';
TABLE: 'TABLE';
ALTER: 'ALTER';
KILL: 'KILL';

ADD: 'ADD';
COLUMN: 'COLUMN';
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ statement returns [Statement value]
| refresh { $value = $refresh.value; }
| set { $value = $set.value; }
| resetStatement { $value = $resetStatement.value; }
| killStatement { $value = $killStatement.value; }
;

query returns [Query value]
Expand Down Expand Up @@ -961,3 +962,7 @@ set returns [SetStatement value]
resetStatement returns [ResetStatement value]
: ^(RESET columns=columnList) { $value = new ResetStatement($columns.value); }
;

killStatement returns [KillStatement value]
: ^(KILL ALL) { $value = KillStatement.INSTANCE; }
;
3 changes: 3 additions & 0 deletions sql-parser/src/main/java/io/crate/sql/tree/AstVisitor.java
Original file line number Diff line number Diff line change
Expand Up @@ -583,4 +583,7 @@ public R visitMatchPredicateColumnIdent(MatchPredicateColumnIdent node, C contex
return visitExpression(node, context);
}

public R visitKillStatement(KillStatement node, C context) {
return visitStatement(node, context);
}
}
49 changes: 49 additions & 0 deletions sql-parser/src/main/java/io/crate/sql/tree/KillStatement.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
/*
* 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.sql.tree;

public class KillStatement extends Statement {

public static final KillStatement INSTANCE = new KillStatement();

private KillStatement() {}

@Override
public int hashCode() {
return System.identityHashCode(this);
}

@Override
public boolean equals(Object obj) {
return this == obj;
}

@Override
public String toString() {
return "KILL ALL";
}

@Override
public <R, C> R accept(AstVisitor<R, C> visitor, C context) {
return visitor.visitKillStatement(this, context);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import static org.hamcrest.core.Is.is;
import static org.hamcrest.core.IsInstanceOf.instanceOf;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
Expand Down Expand Up @@ -233,6 +234,7 @@ public void testStatementBuilder()
printStatement("insert into t (a, b) values (1, 2) on duplicate key update a = a + 1, b = 3");
printStatement("insert into t (a, b) values (1, 2), (3, 4) on duplicate key update a = values (a) + 1, b = 4");
printStatement("insert into t (a, b) values (1, 2), (3, 4) on duplicate key update a = values (a) + 1, b = values(b) - 2");
printStatement("kill all");
}

@Test
Expand Down Expand Up @@ -494,6 +496,12 @@ public Object visitParameterExpression(ParameterExpression node, Object context)
assertEquals(3, counter.get());
}

@Test
public void testKill() throws Exception {
Statement stmt = SqlParser.createStatement("KILL ALL");
assertTrue("stmt not identical to singleton", stmt == KillStatement.INSTANCE);
}

private static void printStatement(String sql)
{
println(sql.trim());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,4 +100,8 @@ public R visitSetStatement(SetAnalyzedStatement analysis, C context) {
public R visitAddColumnStatement(AddColumnAnalyzedStatement analysis, C context) {
return visitDDLAnalyzedStatement(analysis, context);
}

public R visitKillAnalyzedStatement(KillAnalyzedStatement analysis, C context) {
return visitAnalyzedStatement(analysis, context);
}
}
5 changes: 5 additions & 0 deletions sql/src/main/java/io/crate/analyze/Analyzer.java
Original file line number Diff line number Diff line change
Expand Up @@ -185,6 +185,11 @@ public AnalyzedStatement visitResetStatement(ResetStatement node, Analysis conte
return setStatementAnalyzer.analyze(node, context);
}

@Override
public AnalyzedStatement visitKillStatement(KillStatement node, Analysis context) {
return KillAnalyzedStatement.INSTANCE;
}

@Override
protected AnalyzedStatement visitNode(Node node, Analysis context) {
throw new UnsupportedOperationException(String.format("cannot analyze statement: '%s'", node));
Expand Down
35 changes: 35 additions & 0 deletions sql/src/main/java/io/crate/analyze/KillAnalyzedStatement.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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.analyze;

public class KillAnalyzedStatement implements AnalyzedStatement {

public static final KillAnalyzedStatement INSTANCE = new KillAnalyzedStatement();

private KillAnalyzedStatement() {
}

@Override
public <C, R> R accept(AnalyzedStatementVisitor<C, R> analyzedStatementVisitor, C context) {
return analyzedStatementVisitor.visitKillAnalyzedStatement(this, context);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import io.crate.planner.node.ddl.*;
import io.crate.planner.node.dml.*;
import io.crate.planner.node.dql.*;
import io.crate.planner.node.management.KillPlan;
import org.elasticsearch.action.bulk.BulkRetryCoordinatorPool;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.common.Nullable;
Expand Down Expand Up @@ -318,6 +319,11 @@ public List<Task> visitInsertByQuery(InsertFromSubQuery node, Job job) {
public List<Task> visitQueryThenFetch(QueryThenFetch plan, Job job) {
return ImmutableList.of(createExecutableNodesTask(job, plan.collectNode(), plan.mergeNode()));
}

@Override
public List<Task> visitKillPlan(KillPlan killPlan, Job job) {
return ImmutableList.<Task>of(new KillTask(job.id()));
}
}

class NodeVisitor extends PlanNodeVisitor<UUID, ImmutableList<Task>> {
Expand Down
58 changes: 58 additions & 0 deletions sql/src/main/java/io/crate/executor/transport/task/KillTask.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* 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.task;

import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import io.crate.executor.JobTask;
import io.crate.executor.TaskResult;

import java.util.List;
import java.util.UUID;

public class KillTask extends JobTask {

// TODO: add kill transport
public KillTask(UUID jobId) {
super(jobId);
}

@Override
public void start() {
// TODO: do something
}

@Override
public List<ListenableFuture<TaskResult>> result() {
return ImmutableList.of(
Futures.<TaskResult>immediateFailedFuture(
new UnsupportedOperationException("KILL statement not supported")
)
);
}

@Override
public void upstreamResult(List<ListenableFuture<TaskResult>> result) {
// ignore
}
}
5 changes: 5 additions & 0 deletions sql/src/main/java/io/crate/planner/PlanVisitor.java
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import io.crate.planner.node.dml.InsertFromSubQuery;
import io.crate.planner.node.dql.*;
import io.crate.planner.node.dml.Upsert;
import io.crate.planner.node.management.KillPlan;
import org.elasticsearch.common.Nullable;

public class PlanVisitor<C, R> {
Expand Down Expand Up @@ -79,4 +80,8 @@ public R visitCollectAndMerge(CollectAndMerge plan, C context) {
public R visitCountPlan(CountPlan countPlan, C context) {
return visitPlan(countPlan, context);
}

public R visitKillPlan(KillPlan killPlan, C context) {
return visitPlan(killPlan, context);
}
}
6 changes: 6 additions & 0 deletions sql/src/main/java/io/crate/planner/Planner.java
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@
import io.crate.planner.node.dql.CollectNode;
import io.crate.planner.node.dql.FileUriCollectNode;
import io.crate.planner.node.dql.MergeNode;
import io.crate.planner.node.management.KillPlan;
import io.crate.planner.projection.AggregationProjection;
import io.crate.planner.projection.Projection;
import io.crate.planner.projection.SourceIndexWriterProjection;
Expand Down Expand Up @@ -466,6 +467,11 @@ public Plan visitSetStatement(SetAnalyzedStatement analysis, Context context) {
return node != null ? new IterablePlan(node) : NoopPlan.INSTANCE;
}

@Override
public Plan visitKillAnalyzedStatement(KillAnalyzedStatement analysis, Context context) {
return KillPlan.INSTANCE;
}

private void createESDeleteNode(TableInfo tableInfo, WhereClause whereClause, IterablePlan plan) {
plan.add(new ESDeleteNode(tableInfo, whereClause.docKeys().get().getOnlyKey()));
}
Expand Down
38 changes: 38 additions & 0 deletions sql/src/main/java/io/crate/planner/node/management/KillPlan.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* 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.planner.node.management;

import io.crate.planner.Plan;
import io.crate.planner.PlanVisitor;

public class KillPlan implements Plan {

public static final KillPlan INSTANCE = new KillPlan();

private KillPlan() {
}

@Override
public <C, R> R accept(PlanVisitor<C, R> visitor, C context) {
return visitor.visitKillPlan(this, context);
}
}
Loading

0 comments on commit 84abe9a

Please sign in to comment.