Skip to content

Commit

Permalink
feat: add an initial set of execution steps (#3214)
Browse files Browse the repository at this point in the history
This patch adds an initial set of execution step nodes. Subsequent patches
will have schemakstream/table build the execution tree as it goes along, and
then we'll move calls to streams into the implementations of ExecutionStep.build.

All execution steps implement ExecutionStep, which along with supporting a few
common properties (step id and schema), includes a method called build(), which
will eventually get called to build the streams app (as described above).
  • Loading branch information
rodesai committed Aug 15, 2019
1 parent 4a2e4b9 commit c860793
Show file tree
Hide file tree
Showing 32 changed files with 1,499 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import io.confluent.ksql.execution.expression.tree.QualifiedName;
import io.confluent.ksql.execution.expression.tree.QualifiedNameReference;
import io.confluent.ksql.execution.expression.tree.VisitParentExpressionVisitor;
import io.confluent.ksql.execution.plan.SelectExpression;
import io.confluent.ksql.function.AggregateFunctionArguments;
import io.confluent.ksql.function.FunctionRegistry;
import io.confluent.ksql.function.KsqlAggregateFunction;
Expand All @@ -52,7 +53,6 @@
import io.confluent.ksql.util.AggregateExpressionRewriter;
import io.confluent.ksql.util.ExpressionTypeManager;
import io.confluent.ksql.util.KsqlException;
import io.confluent.ksql.util.SelectExpression;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,13 @@

import com.google.common.collect.ImmutableList;
import io.confluent.ksql.execution.expression.tree.Expression;
import io.confluent.ksql.execution.plan.SelectExpression;
import io.confluent.ksql.metastore.model.KeyField;
import io.confluent.ksql.physical.KsqlQueryBuilder;
import io.confluent.ksql.schema.ksql.LogicalSchema;
import io.confluent.ksql.services.KafkaTopicClient;
import io.confluent.ksql.structured.SchemaKStream;
import io.confluent.ksql.util.KsqlException;
import io.confluent.ksql.util.SelectExpression;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,10 @@

import io.confluent.ksql.GenericRow;
import io.confluent.ksql.execution.expression.tree.Expression;
import io.confluent.ksql.execution.plan.SelectExpression;
import io.confluent.ksql.logging.processing.ProcessingLogContext;
import io.confluent.ksql.metastore.model.KeyField;
import io.confluent.ksql.schema.ksql.LogicalSchema;
import io.confluent.ksql.util.SelectExpression;
import java.util.List;
import java.util.Set;
import org.apache.kafka.common.serialization.Serde;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import io.confluent.ksql.execution.expression.tree.DereferenceExpression;
import io.confluent.ksql.execution.expression.tree.Expression;
import io.confluent.ksql.execution.expression.tree.QualifiedNameReference;
import io.confluent.ksql.execution.plan.SelectExpression;
import io.confluent.ksql.function.FunctionRegistry;
import io.confluent.ksql.logging.processing.ProcessingLogContext;
import io.confluent.ksql.logging.processing.ProcessingLogger;
Expand All @@ -40,7 +41,6 @@
import io.confluent.ksql.util.ParserUtil;
import io.confluent.ksql.util.QueryLoggerUtil;
import io.confluent.ksql.util.SchemaUtil;
import io.confluent.ksql.util.SelectExpression;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import com.google.common.collect.ImmutableList;
import io.confluent.ksql.GenericRow;
import io.confluent.ksql.execution.expression.tree.Expression;
import io.confluent.ksql.execution.plan.SelectExpression;
import io.confluent.ksql.function.FunctionRegistry;
import io.confluent.ksql.logging.processing.ProcessingLogContext;
import io.confluent.ksql.metastore.model.KeyField;
Expand All @@ -30,7 +31,6 @@
import io.confluent.ksql.streams.StreamsUtil;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.QueryLoggerUtil;
import io.confluent.ksql.util.SelectExpression;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@
import io.confluent.ksql.structured.QueryContext.Stacker;
import io.confluent.ksql.structured.SchemaKStream;
import io.confluent.ksql.util.KsqlException;
import io.confluent.ksql.util.SelectExpression;
import io.confluent.ksql.execution.plan.SelectExpression;
import java.util.Arrays;
import java.util.Optional;
import org.apache.kafka.connect.data.Schema;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.MetaStoreFixture;
import io.confluent.ksql.util.SchemaUtil;
import io.confluent.ksql.util.SelectExpression;
import io.confluent.ksql.execution.plan.SelectExpression;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.MetaStoreFixture;
import io.confluent.ksql.util.SchemaUtil;
import io.confluent.ksql.util.SelectExpression;
import io.confluent.ksql.execution.plan.SelectExpression;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
import io.confluent.ksql.util.ExpressionMetadata;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.MetaStoreFixture;
import io.confluent.ksql.util.SelectExpression;
import io.confluent.ksql.execution.plan.SelectExpression;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* Copyright 2019 Confluent Inc.
*
* Licensed under the Confluent Community License; you may not use this file
* except in compliance with the License. You may obtain a copy of the License at
*
* http://www.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.execution.plan;

import io.confluent.ksql.schema.ksql.LogicalSchema;
import java.util.Objects;
import jdk.nashorn.internal.ir.annotations.Immutable;

@Immutable
public class DefaultExecutionStepProperties implements ExecutionStepProperties {
private final String id;
private final LogicalSchema schema;

public DefaultExecutionStepProperties(final String id, final LogicalSchema schema) {
this.id = Objects.requireNonNull(id, "id");
this.schema = Objects.requireNonNull(schema, "schema");
}

public LogicalSchema getSchema() {
return schema;
}

public String getId() {
return id;
}

@Override
public boolean equals(final Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final DefaultExecutionStepProperties that = (DefaultExecutionStepProperties) o;
return Objects.equals(id, that.id)
&& Objects.equals(schema, that.schema);
}

@Override
public int hashCode() {
return Objects.hash(id, schema);
}

@Override
public String toString() {
return "ExecutionStepProperties{"
+ "id='" + id + '\''
+ ", schema=" + schema
+ '}';
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* Copyright 2019 Confluent Inc.
*
* Licensed under the Confluent Community License; you may not use this file
* except in compliance with the License. You may obtain a copy of the License at
*
* http://www.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.execution.plan;

import java.util.List;
import org.apache.kafka.streams.StreamsBuilder;

public interface ExecutionStep<T> {
ExecutionStepProperties getProperties();

List<ExecutionStep<?>> getSources();

T build(StreamsBuilder builder);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
/*
* Copyright 2019 Confluent Inc.
*
* Licensed under the Confluent Community License; you may not use this file
* except in compliance with the License. You may obtain a copy of the License at
*
* http://www.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.execution.plan;

import io.confluent.ksql.schema.ksql.LogicalSchema;

public interface ExecutionStepProperties {
LogicalSchema getSchema();

String getId();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* Copyright 2019 Confluent Inc.
*
* Licensed under the Confluent Community License; you may not use this file
* except in compliance with the License. You may obtain a copy of the License at
*
* http://www.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.execution.plan;

import com.google.errorprone.annotations.Immutable;
import io.confluent.ksql.serde.KeyFormat;
import io.confluent.ksql.serde.SerdeOption;
import io.confluent.ksql.serde.ValueFormat;
import java.util.Objects;
import java.util.Set;

@Immutable
public final class Formats {
private final KeyFormat keyFormat;
private final ValueFormat valueFormat;
private final Set<SerdeOption> options;

public Formats(
final KeyFormat keyFormat,
final ValueFormat valueFormat,
final Set<SerdeOption> options) {
this.keyFormat = Objects.requireNonNull(keyFormat, "keyFormat");
this.valueFormat = Objects.requireNonNull(valueFormat, "valueFormat");
this.options = Objects.requireNonNull(options, "options");
}

public KeyFormat getKeyFormat() {
return keyFormat;
}

public ValueFormat getValueFormat() {
return valueFormat;
}

public Set<SerdeOption> getOptions() {
return options;
}

@Override
public boolean equals(final Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final Formats serdeInfo = (Formats) o;
return Objects.equals(keyFormat, serdeInfo.keyFormat)
&& Objects.equals(valueFormat, serdeInfo.valueFormat)
&& Objects.equals(options, serdeInfo.options);
}

@Override
public int hashCode() {
return Objects.hash(keyFormat, valueFormat, options);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,21 @@
/*
* Copyright 2019 Confluent Inc.
*
* Licensed under the Confluent Community License; you may not use this file
* except in compliance with the License. You may obtain a copy of the License at
*
* http://www.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.execution.plan;

public enum JoinType {
INNER,
LEFT,
OUTER
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.util;
package io.confluent.ksql.execution.plan;

import io.confluent.ksql.execution.expression.tree.Expression;
import java.util.Objects;
Expand Down
Loading

0 comments on commit c860793

Please sign in to comment.