diff --git a/Makefile b/Makefile index 933178a..c37c76f 100644 --- a/Makefile +++ b/Makefile @@ -18,40 +18,57 @@ integration-tests: clean: ./gradlew clean -deploy-demo: - kubectl apply -f ./deploy/samples/demodb.yaml +deploy-config: + kubectl create configmap hoptimator-configmap --from-file=model.yaml=test-model.yaml --dry-run=client -o yaml | kubectl apply -f - deploy: deploy-config kubectl apply -f ./hoptimator-k8s/src/main/resources/ kubectl apply -f ./deploy -undeploy: - kubectl delete -f ./deploy || echo "skipping" - kubectl delete configmap hoptimator-configmap || echo "skipping" - quickstart: build deploy -deploy-dev-environment: +deploy-demo: deploy + kubectl apply -f ./deploy/samples/demodb.yaml + +deploy-samples: deploy + kubectl wait --for=condition=Established=True \ + crds/subscriptions.hoptimator.linkedin.com \ + crds/kafkatopics.hoptimator.linkedin.com \ + crds/sqljobs.hoptimator.linkedin.com + kubectl apply -f ./deploy/samples + +deploy-dev-environment: deploy-config kubectl create -f https://github.com/jetstack/cert-manager/releases/download/v1.8.2/cert-manager.yaml || echo "skipping" kubectl create namespace kafka || echo "skipping" helm repo add flink-operator-repo https://downloads.apache.org/flink/flink-kubernetes-operator-1.9.0/ helm upgrade --install --atomic --set webhook.create=false flink-kubernetes-operator flink-operator-repo/flink-kubernetes-operator kubectl apply -f "https://strimzi.io/install/latest?namespace=kafka" -n kafka kubectl wait --for=condition=Established=True crds/kafkas.kafka.strimzi.io + kubectl apply -f ./hoptimator-k8s/src/main/resources/ kubectl apply -f ./deploy/dev + kubectl apply -f ./deploy/samples/demodb.yaml + kubectl apply -f ./deploy/samples/kafkadb.yaml -deploy-samples: deploy - kubectl wait --for=condition=Established=True \ - crds/subscriptions.hoptimator.linkedin.com \ - crds/kafkatopics.hoptimator.linkedin.com \ - crds/sqljobs.hoptimator.linkedin.com - kubectl apply -f ./deploy/samples +undeploy-dev-environment: + kubectl delete $(shell kubectl get kafkatopic.kafka.strimzi.io -o name -n kafka) -n kafka || echo "skipping" + kubectl delete $(shell kubectl get strimzi -o name -n kafka) -n kafka || echo "skipping" + kubectl delete pvc -l strimzi.io/name=one-kafka -n kafka || echo "skipping" + kubectl delete -f "https://strimzi.io/install/latest?namespace=kafka" -n kafka || echo "skipping" + kubectl delete -f ./deploy/samples/kafkadb.yaml || echo "skipping" + kubectl delete -f ./deploy/samples/demodb.yaml || echo "skipping" + kubectl delete -f ./deploy/dev || echo "skipping" + kubectl delete -f ./hoptimator-k8s/src/main/resources/ || echo "skipping" + kubectl delete namespace kafka || echo "skipping" + helm uninstall flink-kubernetes-operator || echo "skipping" + kubectl delete -f https://github.com/jetstack/cert-manager/releases/download/v1.8.2/cert-manager.yaml || echo "skipping" -deploy-config: - kubectl create configmap hoptimator-configmap --from-file=model.yaml=test-model.yaml --dry-run=client -o yaml | kubectl apply -f - +undeploy: undeploy-dev-environment + kubectl delete -f ./deploy || echo "skipping" + kubectl delete configmap hoptimator-configmap || echo "skipping" generate-models: - ./models/generate-models.sh + ./generate-models.sh + ./hoptimator-models/generate-models.sh # <-- marked for deletion release: test -n "$(VERSION)" # MISSING ARG: $$VERSION diff --git a/README.md b/README.md index 12fab71..bb25e84 100644 --- a/README.md +++ b/README.md @@ -39,11 +39,24 @@ Materialized views result in `pipelines`: Hoptimator requires a Kubernetes cluster. To connect from outside a Kubernetes cluster, make sure your `kubectl` is properly configured. +The below setup will install two local demo DBs, ads and profiles. + ``` $ make install # build and install SQL CLI - $ make deploy deploy-demo # install CRDs and K8s objects - $ kubectl port-forward -n kafka svc/one-kafka-external-0 9092 & - $ ./hoptimator + $ make deploy deploy-demo # install demo DB CRDs and K8s objects + $ ./hoptimator # start the SQL CLI + > !intro +``` + +## Set up Kafka & Flink clusters + +The below setup will install a Kafka and Flink cluster within Kubernetes. + +``` + $ make install # build and install SQL CLI + $ make deploy-dev-environment # start local Kafka & Flink setups + $ kubectl port-forward -n kafka svc/one-kafka-external-0 9092 & # forward external Kafka port for use by SQL CLI + $ ./hoptimator # start the SQL CLI > !intro ``` diff --git a/build.gradle b/build.gradle index be80bcc..033cbd5 100644 --- a/build.gradle +++ b/build.gradle @@ -1,6 +1,11 @@ - +plugins { + id("com.github.spotbugs") version "6.0.26" +} subprojects { + apply plugin: 'checkstyle' + apply plugin: 'com.github.spotbugs' + tasks.withType(JavaCompile) { options.release = 8 options.compilerArgs << '-Xlint:deprecation' @@ -9,4 +14,12 @@ subprojects { tasks.withType(Javadoc) { options.addStringOption('Xdoclint:none', '-quiet') } + + spotbugs { + ignoreFailures = true + showProgress = true + reportsDir = file("$buildDir/spotbugs") + includeFilter = file("$rootDir/config/spotbugs/include.xml") + excludeFilter = file("$rootDir/config/spotbugs/exclude.xml") + } } diff --git a/config/checkstyle/checkstyle.xml b/config/checkstyle/checkstyle.xml new file mode 100644 index 0000000..32c5a12 --- /dev/null +++ b/config/checkstyle/checkstyle.xml @@ -0,0 +1,237 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/config/checkstyle/suppressions.xml b/config/checkstyle/suppressions.xml new file mode 100644 index 0000000..1a9612b --- /dev/null +++ b/config/checkstyle/suppressions.xml @@ -0,0 +1,7 @@ + + + + + diff --git a/config/spotbugs/exclude.xml b/config/spotbugs/exclude.xml new file mode 100644 index 0000000..52c8abf --- /dev/null +++ b/config/spotbugs/exclude.xml @@ -0,0 +1,10 @@ + + + + + + + \ No newline at end of file diff --git a/config/spotbugs/include.xml b/config/spotbugs/include.xml new file mode 100644 index 0000000..5f45c43 --- /dev/null +++ b/config/spotbugs/include.xml @@ -0,0 +1,11 @@ + + + + + + + + diff --git a/deploy/dev/kafka.yaml b/deploy/dev/kafka.yaml index d24fbb0..237b920 100644 --- a/deploy/dev/kafka.yaml +++ b/deploy/dev/kafka.yaml @@ -26,6 +26,10 @@ spec: version: 3.8.0 replicas: 1 listeners: + - name: plain + port: 9094 + type: internal + tls: false - name: tls port: 9093 type: internal diff --git a/deploy/subscriptions.crd.yaml b/deploy/subscriptions.crd.yaml new file mode 100644 index 0000000..67943d9 --- /dev/null +++ b/deploy/subscriptions.crd.yaml @@ -0,0 +1,106 @@ +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + name: subscriptions.hoptimator.linkedin.com +spec: + group: hoptimator.linkedin.com + names: + kind: Subscription + listKind: SubscriptionList + plural: subscriptions + singular: subscription + shortNames: + - sub + - subs + preserveUnknownFields: false + scope: Namespaced + versions: + - name: v1alpha1 + served: true + storage: true + schema: + openAPIV3Schema: + description: Hoptimator Subscription + type: object + properties: + apiVersion: + type: string + kind: + type: string + metadata: + type: object + spec: + description: Subscription spec + type: object + properties: + sql: + description: A single SQL query. + type: string + database: + description: The database in which to create the output/sink table. + type: string + hints: + description: Hints to adapters, which may disregard them. + type: object + additionalProperties: + type: string + required: + - sql + - database + status: + description: Filled in by the operator. + type: object + properties: + ready: + description: Whether the subscription is ready to be consumed. + type: boolean + failed: + description: Indicates that the operator was unable to deploy a pipeline for this subscription. + type: boolean + message: + description: Error or success message, for information only. + type: string + sql: + description: The SQL being implemented by this pipeline. + type: string + hints: + description: The hints being used by this pipeline. + type: object + additionalProperties: + type: string + attributes: + description: Physical attributes of the job and sink/output table. + type: object + additionalProperties: + type: string + resources: + description: The yaml generated to implement this pipeline. + type: array + items: + type: string + jobResources: + description: The yaml generated to implement the job. + type: array + items: + type: string + downstreamResources: + description: The yaml generated to implement the sink/output table. + type: array + items: + type: string + subresources: + status: {} + additionalPrinterColumns: + - name: STATUS + type: string + description: Status message from the operator. + jsonPath: .status.message + - name: DB + type: string + description: The database where the subscription is materialized. + jsonPath: .spec.database + - name: SQL + type: string + description: The SQL query that the subscription materializes. + jsonPath: .spec.sql + diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Catalog.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Catalog.java index 9523201..0119525 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Catalog.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Catalog.java @@ -1,12 +1,15 @@ package com.linkedin.hoptimator; -import java.sql.Wrapper; import java.sql.SQLException; +import java.sql.Wrapper; + /** Registers a set of tables, possibly within schemas and sub-schemas. */ public interface Catalog { String name(); + String description(); + void register(Wrapper parentSchema) throws SQLException; } diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/CatalogProvider.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/CatalogProvider.java index 61a9f2a..e5fb87f 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/CatalogProvider.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/CatalogProvider.java @@ -2,6 +2,7 @@ import java.util.Collection; + public interface CatalogProvider { Collection catalogs(); diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Connector.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Connector.java index ff84fe7..25b6678 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Connector.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Connector.java @@ -1,9 +1,10 @@ package com.linkedin.hoptimator; -import java.util.Map; import java.sql.SQLException; +import java.util.Map; + public interface Connector { - + Map configure(T t) throws SQLException; } diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/ConnectorProvider.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/ConnectorProvider.java index 3fca89a..8ddb32c 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/ConnectorProvider.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/ConnectorProvider.java @@ -2,6 +2,7 @@ import java.util.Collection; + public interface ConnectorProvider { Collection> connectors(Class clazz); diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Database.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Database.java index fd86441..f9b1317 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Database.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Database.java @@ -1,6 +1,6 @@ package com.linkedin.hoptimator; -/** A collection of tables, as populated by a Catalog. */ +/** A collection of tables, as populated by a Catalog. */ public interface Database { /** Name of the database. */ diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Deployable.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Deployable.java index 570f421..5be7423 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Deployable.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Deployable.java @@ -1,12 +1,15 @@ package com.linkedin.hoptimator; -import java.util.List; import java.sql.SQLException; +import java.util.List; + public interface Deployable { void create() throws SQLException; + void delete() throws SQLException; + void update() throws SQLException; /** Render a list of specs, usually YAML. */ diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Deployer.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Deployer.java index 8354664..abbe0bf 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Deployer.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Deployer.java @@ -1,12 +1,16 @@ package com.linkedin.hoptimator; -import java.util.List; import java.sql.SQLException; +import java.util.List; + public interface Deployer { - + void create(T t) throws SQLException; + void update(T t) throws SQLException; + void delete(T t) throws SQLException; + List specify(T t) throws SQLException; } diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/DeployerProvider.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/DeployerProvider.java index 8367f1c..341d21a 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/DeployerProvider.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/DeployerProvider.java @@ -2,6 +2,7 @@ import java.util.Collection; + public interface DeployerProvider { Collection> deployers(Class clazz); diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validator.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validator.java index 35118e4..204e8bf 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validator.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validator.java @@ -3,8 +3,8 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -12,7 +12,7 @@ public interface Validator { - + void validate(T t, Issues issues); static void validateSubdomainName(String s, Issues issues) { @@ -142,7 +142,7 @@ private String fullPath() { Collections.reverse(parts); return String.join("/", parts); } - + private String format(int indentLevel) { if (empty()) { return ""; diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/ValidatorProvider.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/ValidatorProvider.java index 8d3935b..95dee2f 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/ValidatorProvider.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/ValidatorProvider.java @@ -2,6 +2,7 @@ import java.util.Collection; + public interface ValidatorProvider { Collection> validators(Class clazz); diff --git a/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroConverter.java b/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroConverter.java index 3370b5a..520b343 100644 --- a/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroConverter.java +++ b/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroConverter.java @@ -1,19 +1,19 @@ package com.linkedin.hoptimator.avro; -import org.apache.avro.Schema; +import java.util.AbstractMap; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.avro.Schema; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rel.type.RelDataTypeImpl; import org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelProtoDataType; -import org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeFactoryImpl; +import org.apache.calcite.sql.type.SqlTypeName; -import java.util.AbstractMap; -import java.util.List; -import java.util.stream.Collectors; /** Converts between Avro and Calcite's RelDataType */ public final class AvroConverter { @@ -23,41 +23,43 @@ private AvroConverter() { public static Schema avro(String namespace, String name, RelDataType dataType) { if (dataType.isStruct()) { - List fields = dataType.getFieldList().stream() - .map(x -> new Schema.Field(sanitize(x.getName()), avro(namespace, x.getName(), x.getType()), describe(x), null)) - .collect(Collectors.toList()); - return createAvroSchemaWithNullability(Schema.createRecord(sanitize(name), dataType.toString(), namespace, false, fields), - dataType.isNullable()); + List fields = dataType.getFieldList() + .stream() + .map(x -> new Schema.Field(sanitize(x.getName()), avro(namespace, x.getName(), x.getType()), describe(x), + null)) + .collect(Collectors.toList()); + return createAvroSchemaWithNullability( + Schema.createRecord(sanitize(name), dataType.toString(), namespace, false, fields), dataType.isNullable()); } else { switch (dataType.getSqlTypeName()) { - case INTEGER: - return createAvroTypeWithNullability(Schema.Type.INT, dataType.isNullable()); - case SMALLINT: - return createAvroTypeWithNullability(Schema.Type.INT, dataType.isNullable()); - case BIGINT: - return createAvroTypeWithNullability(Schema.Type.LONG, dataType.isNullable()); - case VARCHAR: - return createAvroTypeWithNullability(Schema.Type.STRING, dataType.isNullable()); - case FLOAT: - return createAvroTypeWithNullability(Schema.Type.FLOAT, dataType.isNullable()); - case DOUBLE: - return createAvroTypeWithNullability(Schema.Type.DOUBLE, dataType.isNullable()); - case CHAR: - return createAvroTypeWithNullability(Schema.Type.STRING, dataType.isNullable()); - case BOOLEAN: - return createAvroTypeWithNullability(Schema.Type.BOOLEAN, dataType.isNullable()); - case ARRAY: - return createAvroSchemaWithNullability(Schema.createArray(avro(null, null, dataType.getComponentType())), - dataType.isNullable()); - // TODO support map types - // Appears to require a Calcite version bump - // case MAP: - // return createAvroSchemaWithNullability(Schema.createMap(avroPrimitive(dataType.getValueType())), dataType.isNullable()); - case UNKNOWN: - case NULL: - return Schema.createUnion(Schema.create(Schema.Type.NULL)); - default: - throw new UnsupportedOperationException("No support yet for " + dataType.getSqlTypeName().toString()); + case INTEGER: + return createAvroTypeWithNullability(Schema.Type.INT, dataType.isNullable()); + case SMALLINT: + return createAvroTypeWithNullability(Schema.Type.INT, dataType.isNullable()); + case BIGINT: + return createAvroTypeWithNullability(Schema.Type.LONG, dataType.isNullable()); + case VARCHAR: + return createAvroTypeWithNullability(Schema.Type.STRING, dataType.isNullable()); + case FLOAT: + return createAvroTypeWithNullability(Schema.Type.FLOAT, dataType.isNullable()); + case DOUBLE: + return createAvroTypeWithNullability(Schema.Type.DOUBLE, dataType.isNullable()); + case CHAR: + return createAvroTypeWithNullability(Schema.Type.STRING, dataType.isNullable()); + case BOOLEAN: + return createAvroTypeWithNullability(Schema.Type.BOOLEAN, dataType.isNullable()); + case ARRAY: + return createAvroSchemaWithNullability(Schema.createArray(avro(null, null, dataType.getComponentType())), + dataType.isNullable()); + // TODO support map types + // Appears to require a Calcite version bump + // case MAP: + // return createAvroSchemaWithNullability(Schema.createMap(avroPrimitive(dataType.getValueType())), dataType.isNullable()); + case UNKNOWN: + case NULL: + return Schema.createUnion(Schema.create(Schema.Type.NULL)); + default: + throw new UnsupportedOperationException("No support yet for " + dataType.getSqlTypeName().toString()); } } } @@ -82,42 +84,43 @@ private static Schema createAvroTypeWithNullability(Schema.Type rawType, boolean public static RelDataType rel(Schema schema, RelDataTypeFactory typeFactory) { RelDataType unknown = typeFactory.createUnknownType(); switch (schema.getType()) { - case RECORD: - return typeFactory.createStructType(schema.getFields().stream() - .map(x -> new AbstractMap.SimpleEntry<>(x.name(), rel(x.schema(), typeFactory))) - .filter(x -> x.getValue().getSqlTypeName() != SqlTypeName.NULL) - .filter(x -> x.getValue().getSqlTypeName() != unknown.getSqlTypeName()) - .collect(Collectors.toList())); - case INT: - return createRelType(typeFactory, SqlTypeName.INTEGER); - case LONG: - return createRelType(typeFactory, SqlTypeName.BIGINT); - case ENUM: - case FIXED: - case STRING: - return createRelType(typeFactory, SqlTypeName.VARCHAR); - case FLOAT: - return createRelType(typeFactory, SqlTypeName.FLOAT); - case DOUBLE: - return createRelType(typeFactory, SqlTypeName.DOUBLE); - case BOOLEAN: - return createRelType(typeFactory, SqlTypeName.BOOLEAN); - case ARRAY: - return typeFactory.createArrayType(rel(schema.getElementType(), typeFactory), -1); + case RECORD: + return typeFactory.createStructType(schema.getFields() + .stream() + .map(x -> new AbstractMap.SimpleEntry<>(x.name(), rel(x.schema(), typeFactory))) + .filter(x -> x.getValue().getSqlTypeName() != SqlTypeName.NULL) + .filter(x -> x.getValue().getSqlTypeName() != unknown.getSqlTypeName()) + .collect(Collectors.toList())); + case INT: + return createRelType(typeFactory, SqlTypeName.INTEGER); + case LONG: + return createRelType(typeFactory, SqlTypeName.BIGINT); + case ENUM: + case FIXED: + case STRING: + return createRelType(typeFactory, SqlTypeName.VARCHAR); + case FLOAT: + return createRelType(typeFactory, SqlTypeName.FLOAT); + case DOUBLE: + return createRelType(typeFactory, SqlTypeName.DOUBLE); + case BOOLEAN: + return createRelType(typeFactory, SqlTypeName.BOOLEAN); + case ARRAY: + return typeFactory.createArrayType(rel(schema.getElementType(), typeFactory), -1); // TODO support map types // Appears to require a Calcite version bump // case MAP: // return typeFactory.createMapType(typeFactory.createSqlType(SqlTypeName.VARCHAR), rel(schema.getValueType(), typeFactory)); - case UNION: - if (schema.isNullable() && schema.getTypes().size() == 2) { - Schema innerType = schema.getTypes().stream().filter(x -> x.getType() != Schema.Type.NULL).findFirst().get(); - return typeFactory.createTypeWithNullability(rel(innerType, typeFactory), true); - } else { - // TODO support more elaborate union types - return typeFactory.createTypeWithNullability(typeFactory.createUnknownType(), true); - } - default: - return typeFactory.createUnknownType(); + case UNION: + if (schema.isNullable() && schema.getTypes().size() == 2) { + Schema innerType = schema.getTypes().stream().filter(x -> x.getType() != Schema.Type.NULL).findFirst().get(); + return typeFactory.createTypeWithNullability(rel(innerType, typeFactory), true); + } else { + // TODO support more elaborate union types + return typeFactory.createTypeWithNullability(typeFactory.createUnknownType(), true); + } + default: + return typeFactory.createUnknownType(); } } diff --git a/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroTableValidator.java b/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroTableValidator.java index 6064314..5472246 100644 --- a/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroTableValidator.java +++ b/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroTableValidator.java @@ -1,6 +1,8 @@ package com.linkedin.hoptimator.avro; -import com.linkedin.hoptimator.Validator; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; import org.apache.avro.Schema; import org.apache.avro.file.DataFileWriter; @@ -11,16 +13,15 @@ import org.apache.calcite.jdbc.JavaTypeFactoryImpl; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.calcite.schema.Table; import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.schema.Table; + +import com.linkedin.hoptimator.Validator; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.OutputStream; /** Validates that tables follow Avro schema evolution rules. */ class AvroTableValidator implements Validator { - + @Override public void validate(SchemaPlus schema, Issues issues) { try { @@ -55,7 +56,7 @@ private void validate(SchemaPlus schema, Table table, Table originalTable, Issue DataFileWriter dataFileWriter = new DataFileWriter(datumWriter)) { dataFileWriter.create(originalAvroSchema, out); for (Object obj : new RandomData(avroSchema, 1)) { - dataFileWriter.append(obj); + dataFileWriter.append(obj); } } catch (IOException | RuntimeException e) { issues.error("Avro schema evolution error: cannot serialize new records using the existing schema"); diff --git a/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroValidatorProvider.java b/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroValidatorProvider.java index 9af629c..aac09aa 100644 --- a/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroValidatorProvider.java +++ b/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroValidatorProvider.java @@ -1,12 +1,13 @@ package com.linkedin.hoptimator.avro; -import com.linkedin.hoptimator.Validator; -import com.linkedin.hoptimator.ValidatorProvider; +import java.util.Collection; +import java.util.Collections; import org.apache.calcite.schema.SchemaPlus; -import java.util.Collection; -import java.util.Collections; +import com.linkedin.hoptimator.Validator; +import com.linkedin.hoptimator.ValidatorProvider; + /** Provides AvroValidator. */ public class AvroValidatorProvider implements ValidatorProvider { diff --git a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/AvroConverter.java b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/AvroConverter.java index 3b2694e..dd7d1b3 100644 --- a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/AvroConverter.java +++ b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/AvroConverter.java @@ -1,19 +1,19 @@ package com.linkedin.hoptimator.catalog; -import org.apache.avro.Schema; +import java.util.AbstractMap; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.avro.Schema; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rel.type.RelDataTypeImpl; import org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelProtoDataType; -import org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeFactoryImpl; +import org.apache.calcite.sql.type.SqlTypeName; -import java.util.AbstractMap; -import java.util.List; -import java.util.stream.Collectors; /** Converts between Avro and Calcite's RelDataType */ public final class AvroConverter { @@ -23,40 +23,40 @@ private AvroConverter() { public static Schema avro(String namespace, String name, RelDataType dataType) { if (dataType.isStruct()) { - List fields = dataType.getFieldList().stream() - .filter(x -> !x.getName().startsWith("__")) // don't write out hidden fields - .map(x -> new Schema.Field(sanitize(x.getName()), avro(namespace, x.getName(), x.getType()), describe(x), null)) - .collect(Collectors.toList()); - return createAvroSchemaWithNullability(Schema.createRecord(sanitize(name), dataType.toString(), namespace, false, fields), - dataType.isNullable()); + List fields = + dataType.getFieldList().stream().filter(x -> !x.getName().startsWith("__")) // don't write out hidden fields + .map(x -> new Schema.Field(sanitize(x.getName()), avro(namespace, x.getName(), x.getType()), describe(x), + null)).collect(Collectors.toList()); + return createAvroSchemaWithNullability( + Schema.createRecord(sanitize(name), dataType.toString(), namespace, false, fields), dataType.isNullable()); } else { switch (dataType.getSqlTypeName()) { - case INTEGER: - return createAvroTypeWithNullability(Schema.Type.INT, dataType.isNullable()); - case BIGINT: - return createAvroTypeWithNullability(Schema.Type.LONG, dataType.isNullable()); - case VARCHAR: - return createAvroTypeWithNullability(Schema.Type.STRING, dataType.isNullable()); - case FLOAT: - return createAvroTypeWithNullability(Schema.Type.FLOAT, dataType.isNullable()); - case DOUBLE: - return createAvroTypeWithNullability(Schema.Type.DOUBLE, dataType.isNullable()); - case CHAR: - return createAvroTypeWithNullability(Schema.Type.STRING, dataType.isNullable()); - case BOOLEAN: - return createAvroTypeWithNullability(Schema.Type.BOOLEAN, dataType.isNullable()); - case ARRAY: - return createAvroSchemaWithNullability(Schema.createArray(avro(null, null, dataType.getComponentType())), - dataType.isNullable()); - // TODO support map types - // Appears to require a Calcite version bump - // case MAP: - // return createAvroSchemaWithNullability(Schema.createMap(avroPrimitive(dataType.getValueType())), dataType.isNullable()); - case UNKNOWN: - case NULL: - return Schema.createUnion(Schema.create(Schema.Type.NULL)); - default: - throw new UnsupportedOperationException("No support yet for " + dataType.getSqlTypeName().toString()); + case INTEGER: + return createAvroTypeWithNullability(Schema.Type.INT, dataType.isNullable()); + case BIGINT: + return createAvroTypeWithNullability(Schema.Type.LONG, dataType.isNullable()); + case VARCHAR: + return createAvroTypeWithNullability(Schema.Type.STRING, dataType.isNullable()); + case FLOAT: + return createAvroTypeWithNullability(Schema.Type.FLOAT, dataType.isNullable()); + case DOUBLE: + return createAvroTypeWithNullability(Schema.Type.DOUBLE, dataType.isNullable()); + case CHAR: + return createAvroTypeWithNullability(Schema.Type.STRING, dataType.isNullable()); + case BOOLEAN: + return createAvroTypeWithNullability(Schema.Type.BOOLEAN, dataType.isNullable()); + case ARRAY: + return createAvroSchemaWithNullability(Schema.createArray(avro(null, null, dataType.getComponentType())), + dataType.isNullable()); + // TODO support map types + // Appears to require a Calcite version bump + // case MAP: + // return createAvroSchemaWithNullability(Schema.createMap(avroPrimitive(dataType.getValueType())), dataType.isNullable()); + case UNKNOWN: + case NULL: + return Schema.createUnion(Schema.create(Schema.Type.NULL)); + default: + throw new UnsupportedOperationException("No support yet for " + dataType.getSqlTypeName().toString()); } } } @@ -81,42 +81,43 @@ private static Schema createAvroTypeWithNullability(Schema.Type rawType, boolean public static RelDataType rel(Schema schema, RelDataTypeFactory typeFactory) { RelDataType unknown = typeFactory.createUnknownType(); switch (schema.getType()) { - case RECORD: - return typeFactory.createStructType(schema.getFields().stream() - .map(x -> new AbstractMap.SimpleEntry<>(x.name(), rel(x.schema(), typeFactory))) - .filter(x -> x.getValue().getSqlTypeName() != SqlTypeName.NULL) - .filter(x -> x.getValue().getSqlTypeName() != unknown.getSqlTypeName()) - .collect(Collectors.toList())); - case INT: - return createRelType(typeFactory, SqlTypeName.INTEGER); - case LONG: - return createRelType(typeFactory, SqlTypeName.BIGINT); - case ENUM: - case FIXED: - case STRING: - return createRelType(typeFactory, SqlTypeName.VARCHAR); - case FLOAT: - return createRelType(typeFactory, SqlTypeName.FLOAT); - case DOUBLE: - return createRelType(typeFactory, SqlTypeName.DOUBLE); - case BOOLEAN: - return createRelType(typeFactory, SqlTypeName.BOOLEAN); - case ARRAY: - return typeFactory.createArrayType(rel(schema.getElementType(), typeFactory), -1); + case RECORD: + return typeFactory.createStructType(schema.getFields() + .stream() + .map(x -> new AbstractMap.SimpleEntry<>(x.name(), rel(x.schema(), typeFactory))) + .filter(x -> x.getValue().getSqlTypeName() != SqlTypeName.NULL) + .filter(x -> x.getValue().getSqlTypeName() != unknown.getSqlTypeName()) + .collect(Collectors.toList())); + case INT: + return createRelType(typeFactory, SqlTypeName.INTEGER); + case LONG: + return createRelType(typeFactory, SqlTypeName.BIGINT); + case ENUM: + case FIXED: + case STRING: + return createRelType(typeFactory, SqlTypeName.VARCHAR); + case FLOAT: + return createRelType(typeFactory, SqlTypeName.FLOAT); + case DOUBLE: + return createRelType(typeFactory, SqlTypeName.DOUBLE); + case BOOLEAN: + return createRelType(typeFactory, SqlTypeName.BOOLEAN); + case ARRAY: + return typeFactory.createArrayType(rel(schema.getElementType(), typeFactory), -1); // TODO support map types // Appears to require a Calcite version bump // case MAP: // return typeFactory.createMapType(typeFactory.createSqlType(SqlTypeName.VARCHAR), rel(schema.getValueType(), typeFactory)); - case UNION: - if (schema.isNullable() && schema.getTypes().size() == 2) { - Schema innerType = schema.getTypes().stream().filter(x -> x.getType() != Schema.Type.NULL).findFirst().get(); - return typeFactory.createTypeWithNullability(rel(innerType, typeFactory), true); - } else { - // TODO support more elaborate union types - return typeFactory.createTypeWithNullability(typeFactory.createUnknownType(), true); - } - default: - return typeFactory.createUnknownType(); + case UNION: + if (schema.isNullable() && schema.getTypes().size() == 2) { + Schema innerType = schema.getTypes().stream().filter(x -> x.getType() != Schema.Type.NULL).findFirst().get(); + return typeFactory.createTypeWithNullability(rel(innerType, typeFactory), true); + } else { + // TODO support more elaborate union types + return typeFactory.createTypeWithNullability(typeFactory.createUnknownType(), true); + } + default: + return typeFactory.createUnknownType(); } } diff --git a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/ConfigProvider.java b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/ConfigProvider.java index bc3c2b1..fd1459e 100644 --- a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/ConfigProvider.java +++ b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/ConfigProvider.java @@ -4,8 +4,9 @@ import java.util.HashMap; import java.util.Map; import java.util.Optional; -import java.util.stream.Collectors; import java.util.function.Function; +import java.util.stream.Collectors; + /** Provides key-value properties, e.g. for connector configs. */ public interface ConfigProvider { @@ -21,8 +22,7 @@ static ConfigProvider from(Map configs) { if (configs == null) { return empty(); } else { - return x -> configs.entrySet().stream() - .collect(Collectors.toMap(y -> y.getKey(), y -> y.getValue().toString())); + return x -> configs.entrySet().stream().collect(Collectors.toMap(y -> y.getKey(), y -> y.getValue().toString())); } } @@ -66,7 +66,6 @@ default ConfigProvider with(String key, Integer value) { } default ConfigProvider withPrefix(String prefix) { - return x -> config(x).entrySet().stream() - .collect(Collectors.toMap(y -> prefix + y.getKey(), y -> y.getValue())); + return x -> config(x).entrySet().stream().collect(Collectors.toMap(y -> prefix + y.getKey(), y -> y.getValue())); } } diff --git a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/DataType.java b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/DataType.java index 3e5e4b5..67a9106 100644 --- a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/DataType.java +++ b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/DataType.java @@ -1,5 +1,8 @@ package com.linkedin.hoptimator.catalog; +import java.util.Collections; +import java.util.stream.Collectors; + import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeField; @@ -8,8 +11,6 @@ import org.apache.calcite.sql.type.SqlTypeFactoryImpl; import org.apache.calcite.sql.type.SqlTypeName; -import java.util.Collections; -import java.util.stream.Collectors; /** Common data types. Not authoratitive or exhaustive. */ public enum DataType { @@ -22,7 +23,7 @@ public enum DataType { private final RelProtoDataType protoType; DataType(RelProtoDataType protoType) { - this.protoType = protoType; + this.protoType = protoType; } public RelProtoDataType proto() { @@ -38,7 +39,7 @@ public RelDataType rel() { return null; } else { return protoType.apply(DEFAULT_TYPE_FACTORY); - } + } } public static RelDataTypeFactory.Builder builder() { @@ -87,9 +88,8 @@ default Struct drop(String name) { return x -> { RelDataType dataType = apply(x); RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(x); - builder.addAll(dataType.getFieldList().stream() - .filter(y -> !y.getName().equals(name)) - .collect(Collectors.toList())); + builder.addAll( + dataType.getFieldList().stream().filter(y -> !y.getName().equals(name)).collect(Collectors.toList())); return builder.build(); }; } @@ -98,9 +98,10 @@ default Struct dropNestedRows() { return x -> { RelDataType dataType = apply(x); RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(x); - builder.addAll(dataType.getFieldList().stream() - .filter(y -> y.getType().getSqlTypeName() != SqlTypeName.ROW) - .collect(Collectors.toList())); + builder.addAll(dataType.getFieldList() + .stream() + .filter(y -> y.getType().getSqlTypeName() != SqlTypeName.ROW) + .collect(Collectors.toList())); return builder.build(); }; } @@ -116,7 +117,7 @@ default Struct get(String name) { return builder.build(); } }; - } + } } } - + diff --git a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/Database.java b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/Database.java index b1397e5..4fc68c8 100644 --- a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/Database.java +++ b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/Database.java @@ -1,12 +1,13 @@ package com.linkedin.hoptimator.catalog; -import org.apache.calcite.rel.type.RelDataType; - -import java.util.Map; import java.util.Collection; import java.util.Collections; +import java.util.Map; import java.util.concurrent.ExecutionException; +import org.apache.calcite.rel.type.RelDataType; + + /** A set of tables with unique names */ public class Database { private final String name; @@ -22,20 +23,18 @@ public Database(String name, TableLister tableLister, TableResolver tableResolve } /** Convenience constructor for simple connector-based tables */ - public Database(String name, TableLister lister, TableResolver resolver, - ConfigProvider configs) { + public Database(String name, TableLister lister, TableResolver resolver, ConfigProvider configs) { this(name, lister, resolver, TableFactory.connector(configs)); } /** Convenience constructor for simple connector-based tables with resources */ - public Database(String name, TableLister lister, TableResolver resolver, - ConfigProvider configs, ResourceProvider resources) { + public Database(String name, TableLister lister, TableResolver resolver, ConfigProvider configs, + ResourceProvider resources) { this(name, lister, resolver, TableFactory.connector(configs, resources)); } /** Convenience constructor for a list of connector-based tables */ - public Database(String name, Collection tables, TableResolver resolver, - ConfigProvider configs) { + public Database(String name, Collection tables, TableResolver resolver, ConfigProvider configs) { this(name, tables, resolver, TableFactory.connector(configs)); } @@ -46,8 +45,8 @@ public Database(String name, Collection tables, TableResolver resolver, /** Convenience constructor for a static table map */ public Database(String name, Map tableMap) { - this(name, () -> Collections.unmodifiableCollection(tableMap.keySet()), - x -> tableMap.get(x).rowType(), (x, y, z) -> tableMap.get(y)); + this(name, () -> Collections.unmodifiableCollection(tableMap.keySet()), x -> tableMap.get(x).rowType(), + (x, y, z) -> tableMap.get(y)); } /** Find a specific table in the database. */ diff --git a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/DatabaseSchema.java b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/DatabaseSchema.java index f48eb14..42bd514 100644 --- a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/DatabaseSchema.java +++ b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/DatabaseSchema.java @@ -1,10 +1,11 @@ package com.linkedin.hoptimator.catalog; +import java.util.Map; +import java.util.stream.Collectors; + import org.apache.calcite.schema.Table; import org.apache.calcite.schema.impl.AbstractSchema; -import java.util.Map; -import java.util.stream.Collectors; /** Exposes a Database to Apache Calcite. */ public class DatabaseSchema extends AbstractSchema { @@ -14,8 +15,7 @@ public class DatabaseSchema extends AbstractSchema { public DatabaseSchema(Database database) { this.database = database; try { - this.tableMap = database.tables().stream() - .collect(Collectors.toMap(x -> x, x -> new ProtoTable(x, database))); + this.tableMap = database.tables().stream().collect(Collectors.toMap(x -> x, x -> new ProtoTable(x, database))); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopRel.java b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopRel.java index 4f76c2e..103a431 100644 --- a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopRel.java +++ b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopRel.java @@ -3,6 +3,7 @@ import org.apache.calcite.plan.Convention; import org.apache.calcite.rel.RelNode; + /** * Calling convention that ultimately gets converted to a Pipeline or similar. * @@ -11,7 +12,7 @@ * and/or Hoptimator may support additional calling conventions (e.g. batch jobs * or build-and-push jobs). This calling convention -- and indeed, this entire * module -- makes no assumptions about the compute layer. - * + * */ public interface HopRel extends RelNode { Convention CONVENTION = new Convention.Impl("HOP", HopRel.class); diff --git a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopTable.java b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopTable.java index 29a6e3c..47f1290 100644 --- a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopTable.java +++ b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopTable.java @@ -1,18 +1,19 @@ package com.linkedin.hoptimator.catalog; -import org.apache.calcite.schema.impl.AbstractTable; -import org.apache.calcite.schema.TranslatableTable; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; + +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.schema.TranslatableTable; +import org.apache.calcite.schema.impl.AbstractTable; import org.apache.calcite.sql.SqlWriter; import org.apache.calcite.sql.pretty.SqlPrettyWriter; -import org.apache.calcite.plan.RelOptTable; -import org.apache.calcite.plan.RelOptCluster; -import java.util.Collection; -import java.util.Collections; -import java.util.Map; /** * HopTables can have "baggage", including Resources and arbitrary DDL/SQL. @@ -22,7 +23,7 @@ * table can bring along a CDC stream or a cache. Generally, such Resources * won't physically exist until they are needed by a pipeline, at which point * Hoptimator will orchestrate their deployment. - */ + */ public class HopTable extends AbstractTable implements ScriptImplementor, TranslatableTable { private final String database; private final String name; @@ -31,9 +32,8 @@ public class HopTable extends AbstractTable implements ScriptImplementor, Transl private final Collection writeResources; private final ScriptImplementor implementor; - public HopTable(String database, String name, RelDataType rowType, - Collection readResources, Collection writeResources, - ScriptImplementor implementor) { + public HopTable(String database, String name, RelDataType rowType, Collection readResources, + Collection writeResources, ScriptImplementor implementor) { this.database = database; this.name = name; this.rowType = rowType; @@ -43,11 +43,10 @@ public HopTable(String database, String name, RelDataType rowType, } /** Convenience constructor for HopTables that only need a connector config. */ - public HopTable(String database, String name, RelDataType rowType, - Collection readResources, Collection writeResources, - Map connectorConfig) { + public HopTable(String database, String name, RelDataType rowType, Collection readResources, + Collection writeResources, Map connectorConfig) { this(database, name, rowType, readResources, writeResources, - new ScriptImplementor.ConnectorImplementor(database, name, rowType, connectorConfig)); + new ScriptImplementor.ConnectorImplementor(database, name, rowType, connectorConfig)); } /** Convenience constructor for HopTables that only need a connector config. */ @@ -56,10 +55,8 @@ public HopTable(String database, String name, RelDataType rowType, Collection connectorConfig) { + public HopTable(String database, String name, RelDataType rowType, Map connectorConfig) { this(database, name, rowType, Collections.emptyList(), connectorConfig); } diff --git a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopTableScan.java b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopTableScan.java index 33936cc..939d9b8 100644 --- a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopTableScan.java +++ b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopTableScan.java @@ -1,12 +1,13 @@ package com.linkedin.hoptimator.catalog; -import org.apache.calcite.rel.core.TableScan; -import org.apache.calcite.plan.RelTraitSet; -import org.apache.calcite.plan.RelOptTable; +import java.util.Collections; + import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.core.TableScan; -import java.util.Collections; /** Internal. */ public final class HopTableScan extends TableScan implements HopRel { diff --git a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopTableScanRule.java b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopTableScanRule.java index 7ac54be..511d5ce 100644 --- a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopTableScanRule.java +++ b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/HopTableScanRule.java @@ -1,20 +1,18 @@ package com.linkedin.hoptimator.catalog; import org.apache.calcite.plan.Convention; -import org.apache.calcite.plan.RelOptPlanner; -import org.apache.calcite.plan.RelRule; import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.convert.ConverterRule; import org.apache.calcite.rel.logical.LogicalTableScan; + class HopTableScanRule extends ConverterRule { - public static final HopTableScanRule INSTANCE = Config.INSTANCE - .withConversion(LogicalTableScan.class, Convention.NONE, - HopRel.CONVENTION, "HopTableScanRule") - .withRuleFactory(HopTableScanRule::new) - .as(Config.class) - .toRule(HopTableScanRule.class); + public static final HopTableScanRule INSTANCE = + Config.INSTANCE.withConversion(LogicalTableScan.class, Convention.NONE, HopRel.CONVENTION, "HopTableScanRule") + .withRuleFactory(HopTableScanRule::new) + .as(Config.class) + .toRule(HopTableScanRule.class); protected HopTableScanRule(Config config) { super(config); diff --git a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/Names.java b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/Names.java index a637171..c2ab051 100644 --- a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/Names.java +++ b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/Names.java @@ -2,6 +2,7 @@ import java.util.Locale; + public final class Names { private Names() { @@ -10,10 +11,9 @@ private Names() { /** Attempt to format s as a K8s object name, or part of one. */ public static String canonicalize(String s) { return s.toLowerCase(Locale.ROOT) - .replaceAll("[^a-z0-9\\-]+", "-") - .replaceAll("^[^a-z0-9]*", "") - .replaceAll("[^a-z0-9]*$", "") - .replaceAll("\\-+", "-"); + .replaceAll("[^a-z0-9\\-]+", "-") + .replaceAll("^[^a-z0-9]*", "") + .replaceAll("[^a-z0-9]*$", "") + .replaceAll("\\-+", "-"); } - } diff --git a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/ProtoTable.java b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/ProtoTable.java index 7f2d234..4973b93 100644 --- a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/ProtoTable.java +++ b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/ProtoTable.java @@ -1,14 +1,15 @@ package com.linkedin.hoptimator.catalog; -import org.apache.calcite.schema.TranslatableTable; -import org.apache.calcite.schema.impl.AbstractTable; -import org.apache.calcite.plan.RelOptTable; +import java.util.concurrent.ExecutionException; + import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.schema.TranslatableTable; +import org.apache.calcite.schema.impl.AbstractTable; -import java.util.concurrent.ExecutionException; /** Enables lazy-loading of HopTables */ public class ProtoTable extends AbstractTable implements TranslatableTable { diff --git a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/Resource.java b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/Resource.java index 3003098..f585a0c 100644 --- a/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/Resource.java +++ b/hoptimator-catalog/src/main/java/com/linkedin/hoptimator/catalog/Resource.java @@ -1,8 +1,8 @@ package com.linkedin.hoptimator.catalog; -import java.io.InputStream; import java.io.IOException; -import java.util.Arrays; +import java.io.InputStream; +import java.net.URL; import java.util.ArrayList; import java.util.Collection; import java.util.Enumeration; @@ -11,16 +11,15 @@ import java.util.Locale; import java.util.Map; import java.util.Properties; -import java.util.Set; import java.util.Scanner; +import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; -import java.util.UUID; import java.util.function.Supplier; -import java.util.regex.Pattern; import java.util.regex.Matcher; +import java.util.regex.Pattern; import java.util.stream.Collectors; -import java.net.URL; + /** * Represents something required by a Table. @@ -77,8 +76,8 @@ protected void export(String key, String value) { /** Export a map of values */ protected void export(String key, Map values) { - export(key, values.entrySet().stream().map(x -> x.getKey() + ": " + x.getValue()) - .collect(Collectors.joining("\n"))); + export(key, + values.entrySet().stream().map(x -> x.getKey() + ": " + x.getValue()).collect(Collectors.joining("\n"))); } /** Export a list of values */ @@ -124,7 +123,7 @@ public Collection inputs() { @Override public int hashCode() { return toString().hashCode(); - } + } @Override public String toString() { @@ -199,7 +198,7 @@ public SimpleEnvironment with(String key, String value) { Map newVars = new HashMap<>(); newVars.putAll(vars); newVars.put(key, value); - return new SimpleEnvironment(){{ + return new SimpleEnvironment() {{ exportAll(newVars); }}; } @@ -300,12 +299,12 @@ public SimpleTemplate(Environment env, String template) { this.env = env; this.template = template; } - + @Override public String render(Resource resource) { StringBuffer sb = new StringBuffer(); - Pattern p = Pattern.compile( - "([\\s\\-\\#]*)\\{\\{\\s*([\\w_\\-\\.]+)\\s*(:([\\w_\\-\\.]+))?\\s*((\\w+\\s*)*)\\s*\\}\\}"); + Pattern p = + Pattern.compile("([\\s\\-\\#]*)\\{\\{\\s*([\\w_\\-\\.]+)\\s*(:([\\w_\\-\\.]+))?\\s*((\\w+\\s*)*)\\s*\\}\\}"); Matcher m = p.matcher(template); while (m.find()) { String prefix = m.group(1); @@ -325,7 +324,7 @@ public String render(Resource resource) { String replacement = quotedPrefix + quotedValue.replaceAll("\\n", quotedPrefix); m.appendReplacement(sb, replacement); } - m.appendTail(sb); + m.appendTail(sb); return sb.toString(); } @@ -334,18 +333,18 @@ private static String applyTransform(String value, String transform) { String[] funcs = transform.split("\\W+"); for (String f : funcs) { switch (f) { - case "toLowerCase": - res = res.toLowerCase(Locale.ROOT); - break; - case "toUpperCase": - res = res.toUpperCase(Locale.ROOT); - break; - case "toName": - res = canonicalizeName(res); - break; - case "concat": - res = res.replace("\n", ""); - break; + case "toLowerCase": + res = res.toLowerCase(Locale.ROOT); + break; + case "toUpperCase": + res = res.toUpperCase(Locale.ROOT); + break; + case "toName": + res = canonicalizeName(res); + break; + case "concat": + res = res.replace("\n", ""); + break; } } return res; @@ -362,12 +361,11 @@ public interface TemplateFactory { Collection