From 11981d2eaa2f61e81251104d6051acf6fd3911b3 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Fri, 24 Jul 2015 13:20:15 -0700 Subject: [PATCH] Add copycat-data and copycat-api --- build.gradle | 86 + checkstyle/import-control.xml | 15 + .../kafka/copycat/connector/Connector.java | 120 + .../copycat/connector/ConnectorContext.java | 49 + .../copycat/connector/CopycatRecord.java | 111 + .../apache/kafka/copycat/connector/Task.java | 50 + .../copycat/connector/TopicPartition.java | 100 + .../copycat/errors/CopycatException.java | 36 + .../errors/CopycatRuntimeException.java | 35 + .../kafka/copycat/sink/SinkConnector.java | 38 + .../apache/kafka/copycat/sink/SinkRecord.java | 73 + .../apache/kafka/copycat/sink/SinkTask.java | 68 + .../kafka/copycat/sink/SinkTaskContext.java | 42 + .../kafka/copycat/source/SourceConnector.java | 27 + .../kafka/copycat/source/SourceRecord.java | 106 + .../kafka/copycat/source/SourceTask.java | 60 + .../copycat/source/SourceTaskContext.java | 38 + .../kafka/copycat/storage/Converter.java | 42 + .../copycat/storage/OffsetDeserializer.java | 58 + .../copycat/storage/OffsetSerializer.java | 48 + .../copycat/storage/OffsetStorageReader.java | 60 + .../kafka/copycat/util/ConnectorUtils.java | 64 + .../kafka/copycat/util/StringUtils.java | 45 + .../copycat/connector/ConnectorTest.java | 77 + .../copycat/util/ConnectorUtilsTest.java | 67 + .../kafka/copycat/util/StringUtilsTest.java | 38 + .../apache/kafka/copycat/data/BinaryData.java | 35 + .../copycat/data/DataRuntimeException.java | 36 + .../kafka/copycat/data/DataTypeException.java | 33 + .../kafka/copycat/data/GenericArray.java | 34 + .../kafka/copycat/data/GenericContainer.java | 27 + .../kafka/copycat/data/GenericData.java | 1064 +++++++ .../kafka/copycat/data/GenericEnumSymbol.java | 27 + .../kafka/copycat/data/GenericFixed.java | 26 + .../kafka/copycat/data/GenericRecord.java | 30 + .../copycat/data/GenericRecordBuilder.java | 259 ++ .../kafka/copycat/data/IndexedRecord.java | 31 + .../kafka/copycat/data/ObjectProperties.java | 85 + .../kafka/copycat/data/RecordBuilder.java | 32 + .../kafka/copycat/data/RecordBuilderBase.java | 173 ++ .../org/apache/kafka/copycat/data/Schema.java | 1210 ++++++++ .../kafka/copycat/data/SchemaBuilder.java | 2560 +++++++++++++++++ .../copycat/data/SchemaParseException.java | 32 + .../data/UnresolvedUnionException.java | 40 + .../org/apache/kafka/copycat/data/Utf8.java | 158 + .../kafka/copycat/data/TestGenericData.java | 410 +++ settings.gradle | 3 +- 47 files changed, 7857 insertions(+), 1 deletion(-) create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/connector/Connector.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/connector/ConnectorContext.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/connector/CopycatRecord.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/connector/Task.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/connector/TopicPartition.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/errors/CopycatException.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/errors/CopycatRuntimeException.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkConnector.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkRecord.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkTask.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkTaskContext.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceConnector.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceRecord.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceTask.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceTaskContext.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/storage/Converter.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/storage/OffsetDeserializer.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/storage/OffsetSerializer.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReader.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/util/ConnectorUtils.java create mode 100644 copycat-api/src/main/java/org/apache/kafka/copycat/util/StringUtils.java create mode 100644 copycat-api/src/test/java/org/apache/kafka/copycat/connector/ConnectorTest.java create mode 100644 copycat-api/src/test/java/org/apache/kafka/copycat/util/ConnectorUtilsTest.java create mode 100644 copycat-api/src/test/java/org/apache/kafka/copycat/util/StringUtilsTest.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/BinaryData.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/DataRuntimeException.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/DataTypeException.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericArray.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericContainer.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericData.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericEnumSymbol.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericFixed.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericRecord.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericRecordBuilder.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/IndexedRecord.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/ObjectProperties.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/RecordBuilder.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/RecordBuilderBase.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/Schema.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/SchemaBuilder.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/SchemaParseException.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/UnresolvedUnionException.java create mode 100644 copycat-data/src/main/java/org/apache/kafka/copycat/data/Utf8.java create mode 100644 copycat-data/src/test/java/org/apache/kafka/copycat/data/TestGenericData.java diff --git a/build.gradle b/build.gradle index 0abec26fb2d..8c18eac988d 100644 --- a/build.gradle +++ b/build.gradle @@ -445,3 +445,89 @@ project(':log4j-appender') { } test.dependsOn('checkstyleMain', 'checkstyleTest') } + +project(':copycat-data') { + apply plugin: 'checkstyle' + archivesBaseName = "copycat-data" + + dependencies { + compile project(':clients') + compile "org.slf4j:slf4j-api:1.7.6" + + testCompile 'junit:junit:4.6' + testRuntime "$slf4jlog4j" + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/copycat/data/*" + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom (testCompile) + } + + /* FIXME + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') */ +} + +project(':copycat-api') { + apply plugin: 'checkstyle' + archivesBaseName = "copycat-api" + + dependencies { + compile project(':copycat-data') + compile "org.slf4j:slf4j-api:1.7.6" + + testCompile 'junit:junit:4.6' + testRuntime "$slf4jlog4j" + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/copycat/api/*" + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom (testCompile) + } + + /* FIXME + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') */ +} \ No newline at end of file diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 19e0659ef93..74dc339b8b7 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -106,4 +106,19 @@ + + + + + + + + + + + + + + + diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/connector/Connector.java b/copycat-api/src/main/java/org/apache/kafka/copycat/connector/Connector.java new file mode 100644 index 00000000000..386914d66a6 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/connector/Connector.java @@ -0,0 +1,120 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.connector; + +import org.apache.kafka.copycat.errors.CopycatException; + +import java.util.List; +import java.util.Properties; + +/** + *

+ * Connectors manage integration of Copycat with another system, either as an input that ingests + * data into Kafka or an output that passes data to an external system. Implementations should + * not use this class directly; they should inherit from SourceConnector or SinkConnector. + *

+ *

+ * Connectors have two primary tasks. First, given some configuration, they are responsible for + * creating configurations for a set of {@link Task}s that split up the data processing. For + * example, a database Connector might create Tasks by dividing the set of tables evenly among + * tasks. Second, they are responsible for monitoring inputs for changes that require + * reconfiguration and notifying the Copycat runtime via the ConnectorContext. Continuing the + * previous example, the connector might periodically check for new tables and notify Copycat of + * additions and deletions. Copycat will then request new configurations and update the running + * Tasks. + *

+ */ +public abstract class Connector { + + protected ConnectorContext context; + + /** + * Initialize this connector, using the provided ConnectorContext to notify the runtime of + * input configuration changes. + * @param ctx context object used to interact with the Copycat runtime + */ + public void initialize(ConnectorContext ctx) { + context = ctx; + } + + /** + *

+ * Initialize this connector, using the provided ConnectorContext to notify the runtime of + * input configuration changes and using the provided set of Task configurations. + * This version is only used to recover from failures. + *

+ *

+ * The default implementation ignores the provided simply ignores the provided Task + * configurations. During recovery, Copycat will request an updated set of configurations and + * update the running Tasks appropriately. However, Connectors should implement special + * handling of this case if it will avoid unnecessary changes to running Tasks. + *

+ * + * @param ctx context object used to interact with the Copycat runtime + * @param taskConfigs + */ + public void initialize(ConnectorContext ctx, List taskConfigs) { + context = ctx; + // Ignore taskConfigs. May result in more churn of tasks during recovery if updated configs + // are very different, but reduces the difficulty of implementing a Connector + } + + /** + * Start this Connector. This method will only be called on a clean Connector, i.e. it has + * either just been instantiated and initialized or {@link #stop()} has been invoked. + * + * @param props configuration settings + * @throws CopycatException + */ + public abstract void start(Properties props) throws CopycatException; + + /** + * Reconfigure this Connector. Most implementations will not override this, using the default + * implementation that calls {@link #stop()} followed by {@link #start(Properties)}. + * Implementations only need to override this if they want to handle this process more + * efficiently, e.g. without shutting down network connections to the external system. + * + * @param props new configuration settings + * @throws CopycatException + */ + public void reconfigure(Properties props) throws CopycatException { + stop(); + start(props); + } + + /** + * Returns the Task implementation for this Connector. + */ + public abstract Class getTaskClass(); + + /** + * Returns a set of configurations for Tasks based on the current configuration, + * producing at most count configurations. + * + * @param maxTasks maximum number of configurations to generate + * @return configurations for Tasks + */ + public abstract List getTaskConfigs(int maxTasks); + + /** + * Stop this connector. + * + * @throws CopycatException + */ + public abstract void stop() throws CopycatException; +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/connector/ConnectorContext.java b/copycat-api/src/main/java/org/apache/kafka/copycat/connector/ConnectorContext.java new file mode 100644 index 00000000000..41a6b75232e --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/connector/ConnectorContext.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.connector; + +import java.util.List; + +/** + * ConnectorContext allows Connectors to proactively interact with the Copycat runtime. + */ +public interface ConnectorContext { + /** + * Requests that the runtime reconfigure the Tasks for this source. This should be used to + * indicate to the runtime that something about the input/output has changed (e.g. streams + * added/removed) and the running Tasks will need to be modified. + */ + public void requestTaskReconfiguration(); + + /** + * Get a list of TopicPartitions for the specified topics. This should be used to determine how + * to divide TopicPartitions between child tasks. + * @param topics list of topics to get partitions for + * @return list of all TopicPartitions for the input topics + */ + public abstract List getTopicPartitions(String... topics); + + /** + * Get a list of TopicPartitions for the specified topics. This should be used to determine how + * to divide TopicPartitions between child tasks. + * @param topics list of topics to get partitions for + * @return list of all TopicPartitions for the input topics + */ + public abstract List getTopicPartitions(List topics); + +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/connector/CopycatRecord.java b/copycat-api/src/main/java/org/apache/kafka/copycat/connector/CopycatRecord.java new file mode 100644 index 00000000000..8d67fd24017 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/connector/CopycatRecord.java @@ -0,0 +1,111 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.connector; + +/** + *

+ * Base class for records containing data to be copied to/from Kafka. This corresponds closely to + * Kafka's ProducerRecord and ConsumerRecord classes, and holds the data that may be used by both + * sources and sinks (topic, partition, key, value). Although both implementations include a + * notion of offest, it is not included here because they differ in type. + *

+ *

+ * This class uses type parameters for keys and values. These are provided primarily for + * connector developer convenience. Internally, Copycat will handle any primitive types or + * org.apache.kafka.copycat.data types. + *

+ */ +public class CopycatRecord { + private final String topic; + private final Integer partition; + private final Object key; + private final Object value; + + public CopycatRecord(String topic, Integer partition, Object value) { + this(topic, partition, null, value); + } + + public CopycatRecord(String topic, Integer partition, Object key, Object value) { + this.topic = topic; + this.partition = partition; + this.key = key; + this.value = value; + } + + public String getTopic() { + return topic; + } + + public Integer getPartition() { + return partition; + } + + public Object getKey() { + return key; + } + + public Object getValue() { + return value; + } + + @Override + public String toString() { + return "CopycatRecord{" + + "topic='" + topic + '\'' + + ", partition=" + partition + + ", key=" + key + + ", value=" + value + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + CopycatRecord that = (CopycatRecord) o; + + if (key != null ? !key.equals(that.key) : that.key != null) { + return false; + } + if (partition != null ? !partition.equals(that.partition) : that.partition != null) { + return false; + } + if (topic != null ? !topic.equals(that.topic) : that.topic != null) { + return false; + } + if (value != null ? !value.equals(that.value) : that.value != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() { + int result = topic != null ? topic.hashCode() : 0; + result = 31 * result + (partition != null ? partition.hashCode() : 0); + result = 31 * result + (key != null ? key.hashCode() : 0); + result = 31 * result + (value != null ? value.hashCode() : 0); + return result; + } +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/connector/Task.java b/copycat-api/src/main/java/org/apache/kafka/copycat/connector/Task.java new file mode 100644 index 00000000000..1c5efdd59a4 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/connector/Task.java @@ -0,0 +1,50 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.connector; + +import org.apache.kafka.copycat.errors.CopycatException; + +import java.util.Properties; + +/** + *

+ * Tasks contain the code that actually copies data to/from another system. They receive + * a configuration from their parent Connector, assigning them a fraction of a Copycat job's work. + * The Copycat framework then pushes/pulls data from the Task. The Task must also be able to + * respond to reconfiguration requests. + *

+ *

+ * Task only contains the minimal shared functionality between + * {@link org.apache.kafka.copycat.source.SourceTask} and + * {@link org.apache.kafka.copycat.sink.SinkTask}. + *

+ */ +public interface Task { + /** + * Start the Task + * @param props initial configuration + */ + void start(Properties props); + + /** + * Stop this task. + * + * @throws CopycatException + */ + void stop() throws CopycatException; +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/connector/TopicPartition.java b/copycat-api/src/main/java/org/apache/kafka/copycat/connector/TopicPartition.java new file mode 100644 index 00000000000..a0493d812fc --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/connector/TopicPartition.java @@ -0,0 +1,100 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.connector; + +/** + * A topic name and partition number + */ +public class TopicPartition { + + private int hash = 0; + private final int partition; + private final String topic; + + public TopicPartition(String topic, int partition) { + this.partition = partition; + this.topic = topic; + } + + /** + * Parse the TopicPartition from the string representation. + * @param topicPartition string representation of a TopicPartition + */ + public TopicPartition(String topicPartition) { + int lastDash = topicPartition.lastIndexOf('-'); + if (lastDash < 0) { + throw new IllegalArgumentException("Invalid TopicPartition format"); + } + this.topic = topicPartition.substring(0, lastDash); + String partitionStr = topicPartition.substring(lastDash + 1); + this.partition = Integer.parseInt(partitionStr); + } + + public int partition() { + return partition; + } + + public String topic() { + return topic; + } + + @Override + public int hashCode() { + if (hash != 0) { + return hash; + } + final int prime = 31; + int result = 1; + result = prime * result + partition; + result = prime * result + ((topic == null) ? 0 : topic.hashCode()); + this.hash = result; + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + TopicPartition other = (TopicPartition) obj; + if (partition != other.partition) { + return false; + } + if (topic == null) { + if (other.topic != null) { + return false; + } + } else if (!topic.equals(other.topic)) { + return false; + } + return true; + } + + @Override + public String toString() { + return topic + "-" + partition; + } + +} + diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/errors/CopycatException.java b/copycat-api/src/main/java/org/apache/kafka/copycat/errors/CopycatException.java new file mode 100644 index 00000000000..ac46250f8c8 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/errors/CopycatException.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.errors; + +/** + * CopycatException is the top-level exception type generated by Copycat and connectors. + */ +public class CopycatException extends Exception { + + public CopycatException(String s) { + super(s); + } + + public CopycatException(String s, Throwable throwable) { + super(s, throwable); + } + + public CopycatException(Throwable throwable) { + super(throwable); + } +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/errors/CopycatRuntimeException.java b/copycat-api/src/main/java/org/apache/kafka/copycat/errors/CopycatRuntimeException.java new file mode 100644 index 00000000000..44fc9e7343e --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/errors/CopycatRuntimeException.java @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.errors; + +public class CopycatRuntimeException extends RuntimeException { + public CopycatRuntimeException() { + } + + public CopycatRuntimeException(String s) { + super(s); + } + + public CopycatRuntimeException(String s, Throwable throwable) { + super(s, throwable); + } + + public CopycatRuntimeException(Throwable throwable) { + super(throwable); + } +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkConnector.java b/copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkConnector.java new file mode 100644 index 00000000000..2755d17033e --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkConnector.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.sink; + +import org.apache.kafka.copycat.connector.Connector; + +/** + * SinkConnectors implement the Connector interface to send Kafka data to another system. + */ +public abstract class SinkConnector extends Connector { + + /** + *

+ * Configuration key for the list of input topics for this connector. + *

+ *

+ * Usually this setting is only relevant to the Copycat framework, but is provided here for + * the convenience of Connector developers if they also need to know the set of topics. + *

+ */ + public static final String TOPICS_CONFIG = "topics"; + +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkRecord.java b/copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkRecord.java new file mode 100644 index 00000000000..a1069320aca --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkRecord.java @@ -0,0 +1,73 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.sink; + +import org.apache.kafka.copycat.connector.CopycatRecord; + +/** + * SinkRecord is a CopycatRecord that has been read from Kafka and includes the offset of + * the record in the Kafka topic-partition in addition to the standard fields. This information + * should be used by the SinkTask to coordinate offset commits. + */ +public class SinkRecord extends CopycatRecord { + private final long offset; + + public SinkRecord(String topic, int partition, Object key, Object value, long offset) { + super(topic, partition, key, value); + this.offset = offset; + } + + public long getOffset() { + return offset; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + + SinkRecord that = (SinkRecord) o; + + if (offset != that.offset) { + return false; + } + + return true; + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + (int) (offset ^ (offset >>> 32)); + return result; + } + + @Override + public String toString() { + return "SinkRecord{" + + "offset=" + offset + + "} " + super.toString(); + } +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkTask.java b/copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkTask.java new file mode 100644 index 00000000000..25b88aacde7 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkTask.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ +package org.apache.kafka.copycat.sink; + +import org.apache.kafka.copycat.connector.Task; +import org.apache.kafka.copycat.connector.TopicPartition; +import org.apache.kafka.copycat.errors.CopycatException; + +import java.util.Collection; +import java.util.Map; + +/** + * SinkTask is a Task takes records loaded from Kafka and sends them to another system. In + * addition to the basic {@link #put} interface, SinkTasks must also implement {@link #flush} + * to support offset commits. + */ +public abstract class SinkTask implements Task { + + /** + *

+ * The configuration key that provides the list of topic partitions that are inputs for this + * SinkTask. + *

+ *

+ * Usually this setting is only used by the Copycat framework since it manages the Kafka + * consumer that provides input records. However, it is provided here for the convenience of + * SinkTask implementations that may also want to know the input set of topic partitions. + *

+ */ + public static final String TOPICPARTITIONS_CONFIG = "topic.partitions"; + + protected SinkTaskContext context; + + public void initialize(SinkTaskContext context) { + this.context = context; + } + + /** + * Put the records in the sink. Usually this should send the records to the sink asynchronously + * and immediately return. + * + * @param records the set of records to send + */ + public abstract void put(Collection records) throws CopycatException; + + /** + * Flush all records that have been {@link #put} for the specified topic-partitions. The + * offsets are provided for convenience, but could also be determined by tracking all offsets + * included in the SinkRecords passed to {@link #put}. + * + * @param offsets mapping of TopicPartition to committed offset + */ + public abstract void flush(Map offsets); +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkTaskContext.java b/copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkTaskContext.java new file mode 100644 index 00000000000..aa427c0ae40 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/sink/SinkTaskContext.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.sink; + +import org.apache.kafka.copycat.connector.TopicPartition; + +import java.util.HashMap; +import java.util.Map; + +/** + * Context passed to SinkTasks, allowing them to access utilities in the copycat runtime. + */ +public abstract class SinkTaskContext { + private Map offsets; + + public SinkTaskContext() { + offsets = new HashMap(); + } + + public void resetOffset(Map offsets) { + this.offsets = offsets; + } + + public Map getOffsets() { + return offsets; + } +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceConnector.java b/copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceConnector.java new file mode 100644 index 00000000000..10fd07f4184 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceConnector.java @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ +package org.apache.kafka.copycat.source; + +import org.apache.kafka.copycat.connector.Connector; + +/** + * SourceConnectors implement the connector interface to pull data from another system and send + * it to Kafka. + */ +public abstract class SourceConnector extends Connector { + +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceRecord.java b/copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceRecord.java new file mode 100644 index 00000000000..46238251dc6 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceRecord.java @@ -0,0 +1,106 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.source; + +import org.apache.kafka.copycat.connector.CopycatRecord; + +/** + *

+ * SourceRecords are generated by SourceTasks and passed to Copycat for storage in + * Kafka. In addition to the standard fields in CopycatRecord which specify where data is stored + * in Kafka, they also include a stream and offset. + *

+ *

+ * The stream represents a single input stream that the record came from (e.g. a filename, table + * name, or topic-partition). The offset represents a position in that stream which can be used + * to resume consumption of data. + *

+ *

+ * These values can have arbitrary structure and should be represented using + * org.apache.kafka.copycat.data objects (or primitive values). For example, a database connector + * might specify the stream as a record containing { "db": "database_name", "table": + * "table_name"} and the offset as a Long containing the timestamp of the row. + *

+ */ +public class SourceRecord extends CopycatRecord { + private final Object stream; + private final Object offset; + + public SourceRecord(Object stream, Object offset, String topic, Integer partition, Object value) { + this(stream, offset, topic, partition, null, value); + } + + public SourceRecord(Object stream, Object offset, String topic, Object value) { + this(stream, offset, topic, null, null, value); + } + + public SourceRecord(Object stream, Object offset, String topic, Integer partition, + Object key, Object value) { + super(topic, partition, key, value); + this.stream = stream; + this.offset = offset; + } + + public Object getStream() { + return stream; + } + + public Object getOffset() { + return offset; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + + SourceRecord that = (SourceRecord) o; + + if (offset != null ? !offset.equals(that.offset) : that.offset != null) { + return false; + } + if (stream != null ? !stream.equals(that.stream) : that.stream != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + (stream != null ? stream.hashCode() : 0); + result = 31 * result + (offset != null ? offset.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "SourceRecord{" + + "stream=" + stream + + ", offset=" + offset + + "} " + super.toString(); + } +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceTask.java b/copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceTask.java new file mode 100644 index 00000000000..08dd3b224cf --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceTask.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.source; + +import org.apache.kafka.copycat.connector.Task; + +import java.util.List; + +/** + * SourceTask is a Task that pulls records from another system for storage in Kafka. + */ +public abstract class SourceTask implements Task { + + protected SourceTaskContext context; + + /** + * Initialize this SourceTask with the specified context object. + */ + public void initialize(SourceTaskContext context) { + this.context = context; + } + + /** + * Poll this SourceTask for new records. This method should block if no data is currently + * available. + * + * @return a list of source records + */ + public abstract List poll() throws InterruptedException; + + /** + *

+ * Commit the offsets, up to the offsets that have been returned by {@link #poll()}. This + * method should block until the commit is complete. + *

+ *

+ * SourceTasks are not required to implement this functionality; Copycat will record offsets + * automatically. This hook is provided for systems that also need to store offsets internally + * in their own system. + *

+ */ + public void commit() throws InterruptedException { + // This space intentionally left blank. + } +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceTaskContext.java b/copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceTaskContext.java new file mode 100644 index 00000000000..29e0b3ea9b9 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/source/SourceTaskContext.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ +package org.apache.kafka.copycat.source; + +import org.apache.kafka.copycat.storage.OffsetStorageReader; + +/** + * SourceTaskContext is provided to SourceTasks to allow them to interact with the underlying + * runtime. + */ +public class SourceTaskContext { + private final OffsetStorageReader reader; + + public SourceTaskContext(OffsetStorageReader reader) { + this.reader = reader; + } + + /** + * Get the OffsetStorageReader for this SourceTask. + */ + public OffsetStorageReader getOffsetStorageReader() { + return reader; + } +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/storage/Converter.java b/copycat-api/src/main/java/org/apache/kafka/copycat/storage/Converter.java new file mode 100644 index 00000000000..cb2f4727e77 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/storage/Converter.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + +/** + * The Converter interface provides support for translating between Copycat's runtime data format + * and the "native" runtime format used by the serialization layer. This is used to translate + * two types of data: records and offsets. The (de)serialization is performed by a separate + * component -- the producer or consumer serializer or deserializer for records or a Copycat + * serializer or deserializer for offsets. + */ +public interface Converter { + + /** + * Convert a Copycat data object to a native object for serialization. + * @param value + * @return + */ + Object fromCopycatData(Object value); + + /** + * Convert a native object to a Copycat data object. + * @param value + * @return + */ + Object toCopycatData(Object value); +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/storage/OffsetDeserializer.java b/copycat-api/src/main/java/org/apache/kafka/copycat/storage/OffsetDeserializer.java new file mode 100644 index 00000000000..41a35bb7f31 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/storage/OffsetDeserializer.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + + +import org.apache.kafka.copycat.data.Schema; + +import java.io.Closeable; +import java.util.Map; + +/** + * Deserializer for Copycat offsets. + * @param + */ +public interface OffsetDeserializer extends Closeable { + + /** + * Configure this class. + * @param configs configs in key/value pairs + * @param isKey whether is for key or value + */ + public void configure(Map configs, boolean isKey); + + /** + * Deserialize an offset key or value from the specified connector. + * @param connector connector associated with the data + * @param data serialized bytes + * @return deserialized typed data + */ + public T deserializeOffset(String connector, byte[] data); + + /** + * Deserialize an offset key or value from the specified connector using a schema. + * @param connector connector associated with the data + * @param data serialized bytes + * @param schema schema to deserialize to + * @return deserialized typed data + */ + public T deserializeOffset(String connector, byte[] data, Schema schema); + + @Override + public void close(); +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/storage/OffsetSerializer.java b/copycat-api/src/main/java/org/apache/kafka/copycat/storage/OffsetSerializer.java new file mode 100644 index 00000000000..e34a317c264 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/storage/OffsetSerializer.java @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + +import java.io.Closeable; +import java.util.Map; + +/** + * Serializer for Copycat offsets. + * @param native type of offsets. + */ +public interface OffsetSerializer extends Closeable { + /** + * Configure this class. + * @param configs configs in key/value pairs + * @param isKey whether is for key or value + */ + public void configure(Map configs, boolean isKey); + + /** + * @param connector the connector associated with offsets + * @param data typed data + * @return serialized bytes + */ + public byte[] serializeOffset(String connector, T data); + + /** + * Close this serializer. + */ + @Override + public void close(); + +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReader.java b/copycat-api/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReader.java new file mode 100644 index 00000000000..f1e7862c514 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReader.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + +import org.apache.kafka.copycat.data.Schema; + +import java.util.Collection; +import java.util.Map; + +/** + * OffsetStorageReader provides access to the offset storage used by sources. This can be used by + * connectors to determine offsets to start consuming data from. This is most commonly used during + * initialization of a task, but can also be used during runtime, e.g. when reconfiguring a task. + */ +public interface OffsetStorageReader { + /** + * Get the offset for the specified stream. If the data isn't already available locally, this + * gets it from the backing store, which may require some network round trips. + * + * @param stream object uniquely identifying the stream of data + * @param schema schema used to decode the offset value + * @return object uniquely identifying the offset in the stream of data + */ + public Object getOffset(Object stream, Schema schema); + + /** + *

+ * Get a set of offsets for the specified stream identifiers. This may be more efficient + * than calling {@link #getOffset(Object, Schema)} repeatedly. + *

+ *

+ * Note that when errors occur, this method omits the associated data and tries to return as + * many of the requested values as possible. This allows a task that's managing many streams to + * still proceed with any available data. Therefore, implementations should take care to check + * that the data is actually available in the returned response. The only case when an + * exception will be thrown is if the entire request failed, e.g. because the underlying + * storage was unavailable. + *

+ * + * @param streams set of identifiers for streams of data + * @param schema schema used to decode offset values + * @return a map of stream identifiers to decoded offsets + */ + public Map getOffsets(Collection streams, Schema schema); +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/util/ConnectorUtils.java b/copycat-api/src/main/java/org/apache/kafka/copycat/util/ConnectorUtils.java new file mode 100644 index 00000000000..ee94d37aea1 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/util/ConnectorUtils.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.util; + +import java.util.ArrayList; +import java.util.List; + +/** + * Utilities that connector implementations might find useful. Contains common building blocks + * for writing connectors. + */ +public class ConnectorUtils { + /** + * Given a set of elements and a target number of groups, generates an set of groups of + * elements to match the target number of groups, spreading them evenly among the groups. + * This generates groups with contiguous elements, which results in intuitive ordering if + * your elements are also ordered (e.g. alphabetical sets of table names if you sort + * table names alphabetically to generate the raw partitions) or can result in efficient + * partitioning if elements are sorted according to some criteria that affects performance + * (e.g. topic partitions with the same leader). + * + * @param elements list of elements to partition + * @param numGroups the number of output groups to generate. + */ + public static List> groupPartitions(List elements, int numGroups) { + if (numGroups <= 0) { + throw new IllegalArgumentException("Number of groups must be positive."); + } + + List> result = new ArrayList>(numGroups); + + // Each group has either n+1 or n raw partitions + int perGroup = elements.size() / numGroups; + int leftover = elements.size() - (numGroups * perGroup); + + int assigned = 0; + for (int group = 0; group < numGroups; group++) { + int numThisGroup = group < leftover ? perGroup + 1 : perGroup; + List groupList = new ArrayList(numThisGroup); + for (int i = 0; i < numThisGroup; i++) { + groupList.add(elements.get(assigned)); + assigned++; + } + result.add(groupList); + } + + return result; + } +} diff --git a/copycat-api/src/main/java/org/apache/kafka/copycat/util/StringUtils.java b/copycat-api/src/main/java/org/apache/kafka/copycat/util/StringUtils.java new file mode 100644 index 00000000000..b8981f89f02 --- /dev/null +++ b/copycat-api/src/main/java/org/apache/kafka/copycat/util/StringUtils.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ +package org.apache.kafka.copycat.util; + +/** + * General string utilities that are missing from the standard library and may commonly be + * required by Connector or Task implementations. + */ +public class StringUtils { + + /** + * Generate a String by appending all the @{elements}, converted to Strings, delimited by + * @{delim}. + * @param elements list of elements to concatenate + * @param delim delimiter to place between each element + * @return the concatenated string with delimiters + */ + public static String join(Iterable elements, String delim) { + StringBuilder result = new StringBuilder(); + boolean first = true; + for (T elem : elements) { + if (first) { + first = false; + } else { + result.append(delim); + } + result.append(elem); + } + return result.toString(); + } +} diff --git a/copycat-api/src/test/java/org/apache/kafka/copycat/connector/ConnectorTest.java b/copycat-api/src/test/java/org/apache/kafka/copycat/connector/ConnectorTest.java new file mode 100644 index 00000000000..c6deb48b0f9 --- /dev/null +++ b/copycat-api/src/test/java/org/apache/kafka/copycat/connector/ConnectorTest.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.connector; + +import org.apache.kafka.copycat.errors.CopycatException; +import org.junit.Test; + +import java.util.List; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; + +public class ConnectorTest { + + @Test + public void testDefaultReconfigure() throws Exception { + TestConnector conn = new TestConnector(false); + conn.reconfigure(new Properties()); + assertEquals(conn.stopOrder, 0); + assertEquals(conn.configureOrder, 1); + } + + @Test(expected = CopycatException.class) + public void testReconfigureStopException() throws Exception { + TestConnector conn = new TestConnector(true); + conn.reconfigure(new Properties()); + } + + private static class TestConnector extends Connector { + private boolean stopException; + private int order = 0; + public int stopOrder = -1; + public int configureOrder = -1; + + public TestConnector(boolean stopException) { + this.stopException = stopException; + } + + @Override + public void start(Properties props) throws CopycatException { + configureOrder = order++; + } + + @Override + public Class getTaskClass() { + return null; + } + + @Override + public List getTaskConfigs(int count) { + return null; + } + + @Override + public void stop() throws CopycatException { + stopOrder = order++; + if (stopException) { + throw new CopycatException("error"); + } + } + } +} \ No newline at end of file diff --git a/copycat-api/src/test/java/org/apache/kafka/copycat/util/ConnectorUtilsTest.java b/copycat-api/src/test/java/org/apache/kafka/copycat/util/ConnectorUtilsTest.java new file mode 100644 index 00000000000..e46967b6426 --- /dev/null +++ b/copycat-api/src/test/java/org/apache/kafka/copycat/util/ConnectorUtilsTest.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.util; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class ConnectorUtilsTest { + + private static final List FIVE_ELEMENTS = Arrays.asList(1, 2, 3, 4, 5); + + @Test + public void testGroupPartitions() { + + List> grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 1); + assertEquals(Arrays.asList(FIVE_ELEMENTS), grouped); + + grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 2); + assertEquals(Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5)), grouped); + + grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 3); + assertEquals(Arrays.asList(Arrays.asList(1, 2), + Arrays.asList(3, 4), + Arrays.asList(5)), grouped); + + grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 5); + assertEquals(Arrays.asList(Arrays.asList(1), + Arrays.asList(2), + Arrays.asList(3), + Arrays.asList(4), + Arrays.asList(5)), grouped); + + grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 7); + assertEquals(Arrays.asList(Arrays.asList(1), + Arrays.asList(2), + Arrays.asList(3), + Arrays.asList(4), + Arrays.asList(5), + Collections.EMPTY_LIST, + Collections.EMPTY_LIST), grouped); + } + + @Test(expected = IllegalArgumentException.class) + public void testGroupPartitionsInvalidCount() { + ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 0); + } +} diff --git a/copycat-api/src/test/java/org/apache/kafka/copycat/util/StringUtilsTest.java b/copycat-api/src/test/java/org/apache/kafka/copycat/util/StringUtilsTest.java new file mode 100644 index 00000000000..41d6dcb489a --- /dev/null +++ b/copycat-api/src/test/java/org/apache/kafka/copycat/util/StringUtilsTest.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.util; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; + +import static org.junit.Assert.assertEquals; + +public class StringUtilsTest { + + @Test + public void testJoin() { + assertEquals("", StringUtils.join(Collections.EMPTY_LIST, "")); + assertEquals("", StringUtils.join(Collections.EMPTY_LIST, ",")); + + assertEquals("ab", StringUtils.join(Arrays.asList("a", "b"), "")); + + assertEquals("a,b,c", StringUtils.join(Arrays.asList("a", "b", "c"), ",")); + } +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/BinaryData.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/BinaryData.java new file mode 100644 index 00000000000..3392e5d0d8b --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/BinaryData.java @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.data; + + +public class BinaryData { + public static int compareBytes(byte[] b1, int s1, int l1, + byte[] b2, int s2, int l2) { + int end1 = s1 + l1; + int end2 = s2 + l2; + for (int i = s1, j = s2; i < end1 && j < end2; i++, j++) { + int a = b1[i] & 0xff; + int b = b2[j] & 0xff; + if (a != b) { + return a - b; + } + } + return l1 - l2; + } +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/DataRuntimeException.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/DataRuntimeException.java new file mode 100644 index 00000000000..855c0fd4d79 --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/DataRuntimeException.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +/** Base Avro exception. */ +public class DataRuntimeException extends RuntimeException { + public DataRuntimeException(Throwable cause) { + super(cause); + } + + public DataRuntimeException(String message) { + super(message); + } + + public DataRuntimeException(String message, Throwable cause) { + super(message, cause); + } +} + diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/DataTypeException.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/DataTypeException.java new file mode 100644 index 00000000000..6a74d88103a --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/DataTypeException.java @@ -0,0 +1,33 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + + +/** Thrown when an illegal type is used. */ +public class DataTypeException extends DataRuntimeException { + public DataTypeException(String message) { + super(message); + } + + public DataTypeException(String message, Throwable cause) { + super(message, cause); + } +} + diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericArray.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericArray.java new file mode 100644 index 00000000000..ed708431f65 --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericArray.java @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +import java.util.List; + +/** Array that permits reuse of contained elements. */ +public interface GenericArray extends List, GenericContainer { + /** The current content of the location where {@link #add(Object)} would next + * store an element, if any. This permits reuse of arrays and their elements + * without allocating new objects. */ + T peek(); + + /** Reverses the order of the elements in this array. */ + void reverse(); +} + diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericContainer.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericContainer.java new file mode 100644 index 00000000000..16c43322178 --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericContainer.java @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +/** Contains data of other types. */ +public interface GenericContainer { + /** The schema of this instance. */ + Schema getSchema(); +} + diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericData.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericData.java new file mode 100644 index 00000000000..5c69df2157f --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericData.java @@ -0,0 +1,1064 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +import org.apache.kafka.copycat.data.Schema.Field; +import org.apache.kafka.copycat.data.Schema.Type; + +import java.nio.ByteBuffer; +import java.util.*; + +/** Utilities for generic Java data. See {@link GenericRecordBuilder} for a convenient + * way to build {@link GenericRecord} instances. + * @see GenericRecordBuilder + */ +public class GenericData { + + private static final GenericData INSTANCE = new GenericData(); + + /** Used to specify the Java type for a string schema. */ + public enum StringType { + CharSequence, String, Utf8 + } + + protected static final String STRING_PROP = "avro.java.string"; + protected static final String STRING_TYPE_STRING = "String"; + + private final ClassLoader classLoader; + + /** Set the Java type to be used when reading this schema. Meaningful only + * only string schemas and map schemas (for the keys). */ + public static void setStringType(Schema s, StringType stringType) { + // Utf8 is the default and implements CharSequence, so we only need to add + // a property when the type is String + if (stringType == StringType.String) + s.addProp(GenericData.STRING_PROP, GenericData.STRING_TYPE_STRING); + } + + /** Return the singleton instance. */ + public static GenericData get() { + return INSTANCE; + } + + /** For subclasses. Applications normally use {@link GenericData#get()}. */ + public GenericData() { + this(null); + } + + /** For subclasses. GenericData does not use a ClassLoader. */ + public GenericData(ClassLoader classLoader) { + this.classLoader = (classLoader != null) + ? classLoader + : getClass().getClassLoader(); + } + + /** Return the class loader that's used (by subclasses). */ + public ClassLoader getClassLoader() { + return classLoader; + } + + /** Default implementation of {@link GenericRecord}. Note that this implementation + * does not fill in default values for fields if they are not specified; use {@link + * GenericRecordBuilder} in that case. + * @see GenericRecordBuilder + */ + public static class Record implements GenericRecord, Comparable { + private final Schema schema; + private final Object[] values; + + public Record(Schema schema) { + if (schema == null || !Type.RECORD.equals(schema.getType())) + throw new DataRuntimeException("Not a record schema: " + schema); + this.schema = schema; + this.values = new Object[schema.getFields().size()]; + } + + public Record(Record other, boolean deepCopy) { + schema = other.schema; + values = new Object[schema.getFields().size()]; + if (deepCopy) { + for (int ii = 0; ii < values.length; ii++) { + values[ii] = INSTANCE.deepCopy( + schema.getFields().get(ii).schema(), other.values[ii]); + } + } else { + System.arraycopy(other.values, 0, values, 0, other.values.length); + } + } + + @Override + public Schema getSchema() { + return schema; + } + + @Override + public void put(String key, Object value) { + Schema.Field field = schema.getField(key); + if (field == null) + throw new DataRuntimeException("Not a valid schema field: " + key); + + values[field.pos()] = value; + } + + @Override + public void put(int i, Object v) { + values[i] = v; + } + + @Override + public Object get(String key) { + Field field = schema.getField(key); + if (field == null) return null; + return values[field.pos()]; + } + + @Override + public Object get(int i) { + return values[i]; + } + + @Override + public boolean equals(Object o) { + if (o == this) return true; // identical object + if (!(o instanceof Record)) return false; // not a record + Record that = (Record) o; + if (!this.schema.equals(that.schema)) + return false; // not the same schema + return GenericData.get().compare(this, that, schema, true) == 0; + } + + @Override + public int hashCode() { + return GenericData.get().hashCode(this, schema); + } + + @Override + public int compareTo(Record that) { + return GenericData.get().compare(this, that, schema); + } + + @Override + public String toString() { + return GenericData.get().toString(this); + } + } + + /** Default implementation of an array. */ + @SuppressWarnings(value = "unchecked") + public static class Array extends AbstractList + implements GenericArray, Comparable> { + private static final Object[] EMPTY = new Object[0]; + private final Schema schema; + private int size; + private Object[] elements = EMPTY; + + public Array(int capacity, Schema schema) { + if (schema == null || !Type.ARRAY.equals(schema.getType())) + throw new DataRuntimeException("Not an array schema: " + schema); + this.schema = schema; + if (capacity != 0) + elements = new Object[capacity]; + } + + public Array(Schema schema, Collection c) { + if (schema == null || !Type.ARRAY.equals(schema.getType())) + throw new DataRuntimeException("Not an array schema: " + schema); + this.schema = schema; + if (c != null) { + elements = new Object[c.size()]; + addAll(c); + } + } + + @Override + public Schema getSchema() { + return schema; + } + + @Override + public int size() { + return size; + } + + @Override + public void clear() { + size = 0; + } + + @Override + public Iterator iterator() { + return new Iterator() { + private int position = 0; + + @Override + public boolean hasNext() { + return position < size; + } + + @Override + public T next() { + return (T) elements[position++]; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + + @Override + public T get(int i) { + if (i >= size) + throw new IndexOutOfBoundsException("Index " + i + " out of bounds."); + return (T) elements[i]; + } + + @Override + public boolean add(T o) { + if (size == elements.length) { + Object[] newElements = new Object[(size * 3) / 2 + 1]; + System.arraycopy(elements, 0, newElements, 0, size); + elements = newElements; + } + elements[size++] = o; + return true; + } + + @Override + public void add(int location, T o) { + if (location > size || location < 0) { + throw new IndexOutOfBoundsException("Index " + location + " out of bounds."); + } + if (size == elements.length) { + Object[] newElements = new Object[(size * 3) / 2 + 1]; + System.arraycopy(elements, 0, newElements, 0, size); + elements = newElements; + } + System.arraycopy(elements, location, elements, location + 1, size - location); + elements[location] = o; + size++; + } + + @Override + public T set(int i, T o) { + if (i >= size) + throw new IndexOutOfBoundsException("Index " + i + " out of bounds."); + T response = (T) elements[i]; + elements[i] = o; + return response; + } + + @Override + public T remove(int i) { + if (i >= size) + throw new IndexOutOfBoundsException("Index " + i + " out of bounds."); + T result = (T) elements[i]; + --size; + System.arraycopy(elements, i + 1, elements, i, size - i); + elements[size] = null; + return result; + } + + @Override + public T peek() { + return (size < elements.length) ? (T) elements[size] : null; + } + + @Override + public int compareTo(GenericArray that) { + return GenericData.get().compare(this, that, this.getSchema()); + } + + @Override + public void reverse() { + int left = 0; + int right = elements.length - 1; + + while (left < right) { + Object tmp = elements[left]; + elements[left] = elements[right]; + elements[right] = tmp; + + left++; + right--; + } + } + + @Override + public String toString() { + StringBuilder buffer = new StringBuilder(); + buffer.append("["); + int count = 0; + for (T e : this) { + buffer.append(e == null ? "null" : e.toString()); + if (++count < size()) + buffer.append(", "); + } + buffer.append("]"); + return buffer.toString(); + } + } + + /** Default implementation of {@link GenericFixed}. */ + public static class Fixed implements GenericFixed, Comparable { + private Schema schema; + private byte[] bytes; + + public Fixed(Schema schema) { + setSchema(schema); + } + + public Fixed(Schema schema, byte[] bytes) { + this.schema = schema; + this.bytes = bytes; + } + + protected Fixed() { + } + + protected void setSchema(Schema schema) { + this.schema = schema; + this.bytes = new byte[schema.getFixedSize()]; + } + + @Override + public Schema getSchema() { + return schema; + } + + public void bytes(byte[] bytes) { + this.bytes = bytes; + } + + @Override + public byte[] bytes() { + return bytes; + } + + @Override + public boolean equals(Object o) { + if (o == this) return true; + return o instanceof GenericFixed + && Arrays.equals(bytes, ((GenericFixed) o).bytes()); + } + + @Override + public int hashCode() { + return Arrays.hashCode(bytes); + } + + @Override + public String toString() { + return Arrays.toString(bytes); + } + + @Override + public int compareTo(Fixed that) { + return BinaryData.compareBytes(this.bytes, 0, this.bytes.length, + that.bytes, 0, that.bytes.length); + } + } + + /** Default implementation of {@link GenericEnumSymbol}. */ + public static class EnumSymbol + implements GenericEnumSymbol, Comparable { + private Schema schema; + private String symbol; + + public EnumSymbol(Schema schema, String symbol) { + this.schema = schema; + this.symbol = symbol; + } + + /** + * Maps existing Objects into an Avro enum + * by calling toString(), eg for Java Enums + */ + public EnumSymbol(Schema schema, Object symbol) { + this(schema, symbol.toString()); + } + + @Override + public Schema getSchema() { + return schema; + } + + @Override + public boolean equals(Object o) { + if (o == this) return true; + return o instanceof GenericEnumSymbol + && symbol.equals(o.toString()); + } + + @Override + public int hashCode() { + return symbol.hashCode(); + } + + @Override + public String toString() { + return symbol; + } + + @Override + public int compareTo(GenericEnumSymbol that) { + return GenericData.get().compare(this, that, schema); + } + } + + /** Returns true if a Java datum matches a schema. */ + public boolean validate(Schema schema, Object datum) { + switch (schema.getType()) { + case RECORD: + if (!isRecord(datum)) return false; + for (Field f : schema.getFields()) { + if (!validate(f.schema(), getField(datum, f.name(), f.pos()))) + return false; + } + return true; + case ENUM: + if (!isEnum(datum)) return false; + return schema.getEnumSymbols().contains(datum.toString()); + case ARRAY: + if (!(isArray(datum))) return false; + for (Object element : getArrayAsCollection(datum)) + if (!validate(schema.getElementType(), element)) + return false; + return true; + case MAP: + if (!(isMap(datum))) return false; + @SuppressWarnings(value = "unchecked") + Map map = (Map) datum; + for (Map.Entry entry : map.entrySet()) + if (!validate(schema.getValueType(), entry.getValue())) + return false; + return true; + case UNION: + try { + int i = resolveUnion(schema, datum); + return validate(schema.getTypes().get(i), datum); + } catch (UnresolvedUnionException e) { + return false; + } + case FIXED: + return datum instanceof GenericFixed + && ((GenericFixed) datum).bytes().length == schema.getFixedSize(); + case STRING: + return isString(datum); + case BYTES: + return isBytes(datum); + case INT: + return isInteger(datum); + case LONG: + return isLong(datum); + case FLOAT: + return isFloat(datum); + case DOUBLE: + return isDouble(datum); + case BOOLEAN: + return isBoolean(datum); + case NULL: + return datum == null; + default: + return false; + } + } + + /** Renders a Java datum as JSON. */ + public String toString(Object datum) { + StringBuilder buffer = new StringBuilder(); + toString(datum, buffer); + return buffer.toString(); + } + + /** Renders a Java datum as JSON. */ + protected void toString(Object datum, StringBuilder buffer) { + if (isRecord(datum)) { + buffer.append("{"); + int count = 0; + Schema schema = getRecordSchema(datum); + for (Field f : schema.getFields()) { + toString(f.name(), buffer); + buffer.append(": "); + toString(getField(datum, f.name(), f.pos()), buffer); + if (++count < schema.getFields().size()) + buffer.append(", "); + } + buffer.append("}"); + } else if (isArray(datum)) { + Collection array = getArrayAsCollection(datum); + buffer.append("["); + long last = array.size() - 1; + int i = 0; + for (Object element : array) { + toString(element, buffer); + if (i++ < last) + buffer.append(", "); + } + buffer.append("]"); + } else if (isMap(datum)) { + buffer.append("{"); + int count = 0; + @SuppressWarnings(value = "unchecked") + Map map = (Map) datum; + for (Map.Entry entry : map.entrySet()) { + toString(entry.getKey(), buffer); + buffer.append(": "); + toString(entry.getValue(), buffer); + if (++count < map.size()) + buffer.append(", "); + } + buffer.append("}"); + } else if (isString(datum) || isEnum(datum)) { + buffer.append("\""); + writeEscapedString(datum.toString(), buffer); + buffer.append("\""); + } else if (isBytes(datum)) { + buffer.append("{\"bytes\": \""); + ByteBuffer bytes = (ByteBuffer) datum; + for (int i = bytes.position(); i < bytes.limit(); i++) + buffer.append((char) bytes.get(i)); + buffer.append("\"}"); + } else if (((datum instanceof Float) && // quote Nan & Infinity + (((Float) datum).isInfinite() || ((Float) datum).isNaN())) + || ((datum instanceof Double) && + (((Double) datum).isInfinite() || ((Double) datum).isNaN()))) { + buffer.append("\""); + buffer.append(datum); + buffer.append("\""); + } else { + buffer.append(datum); + } + } + + /* Adapted from http://code.google.com/p/json-simple */ + private void writeEscapedString(String string, StringBuilder builder) { + for (int i = 0; i < string.length(); i++) { + char ch = string.charAt(i); + switch (ch) { + case '"': + builder.append("\\\""); + break; + case '\\': + builder.append("\\\\"); + break; + case '\b': + builder.append("\\b"); + break; + case '\f': + builder.append("\\f"); + break; + case '\n': + builder.append("\\n"); + break; + case '\r': + builder.append("\\r"); + break; + case '\t': + builder.append("\\t"); + break; + default: + // Reference: http://www.unicode.org/versions/Unicode5.1.0/ + if ((ch >= '\u0000' && ch <= '\u001F') || (ch >= '\u007F' && ch <= '\u009F') || (ch >= '\u2000' && ch <= '\u20FF')) { + String hex = Integer.toHexString(ch); + builder.append("\\u"); + for (int j = 0; j < 4 - hex.length(); j++) + builder.append('0'); + builder.append(hex.toUpperCase()); + } else { + builder.append(ch); + } + } + } + } + + /** Called by deep copying implementation to set a record fields + * value to a record instance. The default implementation is for {@link + * IndexedRecord}.*/ + public void setField(Object record, String name, int position, Object o) { + ((IndexedRecord) record).put(position, o); + } + + /** Called by deep copying implementation to set a record + * field value from a reused instance. The default implementation is for + * {@link IndexedRecord}.*/ + public Object getField(Object record, String name, int position) { + return ((IndexedRecord) record).get(position); + } + + /** Produce state for repeated calls to {@link + * #getField(Object, String, int, Object)} and {@link + * #setField(Object, String, int, Object, Object)} on the same record.*/ + protected Object getRecordState(Object record, Schema schema) { + return null; + } + + /** Version of {@link #setField} that has state. */ + protected void setField(Object r, String n, int p, Object o, Object state) { + setField(r, n, p, o); + } + + /** Version of {@link #getField} that has state. */ + protected Object getField(Object record, String name, int pos, Object state) { + return getField(record, name, pos); + } + + /** Return the index for a datum within a union. Implemented with {@link + * Schema#getIndexNamed(String)} and {@link #getSchemaName(Object)}.*/ + public int resolveUnion(Schema union, Object datum) { + Integer i = union.getIndexNamed(getSchemaName(datum)); + if (i != null) + return i; + throw new UnresolvedUnionException(union, datum); + } + + /** Return the schema full name for a datum. Called by {@link + * #resolveUnion(Schema, Object)}. */ + protected String getSchemaName(Object datum) { + if (datum == null) + return Type.NULL.getName(); + if (isRecord(datum)) + return getRecordSchema(datum).getFullName(); + if (isEnum(datum)) + return getEnumSchema(datum).getFullName(); + if (isArray(datum)) + return Type.ARRAY.getName(); + if (isMap(datum)) + return Type.MAP.getName(); + if (isFixed(datum)) + return getFixedSchema(datum).getFullName(); + if (isString(datum)) + return Type.STRING.getName(); + if (isBytes(datum)) + return Type.BYTES.getName(); + if (isInteger(datum)) + return Type.INT.getName(); + if (isLong(datum)) + return Type.LONG.getName(); + if (isFloat(datum)) + return Type.FLOAT.getName(); + if (isDouble(datum)) + return Type.DOUBLE.getName(); + if (isBoolean(datum)) + return Type.BOOLEAN.getName(); + throw new DataRuntimeException(String.format("Unknown datum type %s: %s", datum.getClass().getName(), datum)); + } + + /** Called by {@link #resolveUnion(Schema, Object)}. May be overridden for + alternate data representations.*/ + protected boolean instanceOf(Schema schema, Object datum) { + switch (schema.getType()) { + case RECORD: + if (!isRecord(datum)) return false; + return (schema.getFullName() == null) + ? getRecordSchema(datum).getFullName() == null + : schema.getFullName().equals(getRecordSchema(datum).getFullName()); + case ENUM: + if (!isEnum(datum)) return false; + return schema.getFullName().equals(getEnumSchema(datum).getFullName()); + case ARRAY: + return isArray(datum); + case MAP: + return isMap(datum); + case FIXED: + if (!isFixed(datum)) return false; + return schema.getFullName().equals(getFixedSchema(datum).getFullName()); + case STRING: + return isString(datum); + case BYTES: + return isBytes(datum); + case INT: + return isInteger(datum); + case LONG: + return isLong(datum); + case FLOAT: + return isFloat(datum); + case DOUBLE: + return isDouble(datum); + case BOOLEAN: + return isBoolean(datum); + case NULL: + return datum == null; + default: + throw new DataRuntimeException("Unexpected type: " + schema); + } + } + + /** Called by the default implementation of {@link #instanceOf}.*/ + protected boolean isArray(Object datum) { + return datum instanceof Collection; + } + + /** Called to access an array as a collection. */ + protected Collection getArrayAsCollection(Object datum) { + return (Collection) datum; + } + + /** Called by the default implementation of {@link #instanceOf}.*/ + protected boolean isRecord(Object datum) { + return datum instanceof IndexedRecord; + } + + /** Called to obtain the schema of a record. By default calls + * {GenericContainer#getSchema(). May be overridden for alternate record + * representations. */ + protected Schema getRecordSchema(Object record) { + return ((GenericContainer) record).getSchema(); + } + + /** Called by the default implementation of {@link #instanceOf}.*/ + protected boolean isEnum(Object datum) { + return datum instanceof GenericEnumSymbol; + } + + /** Called to obtain the schema of a enum. By default calls + * {GenericContainer#getSchema(). May be overridden for alternate enum + * representations. */ + protected Schema getEnumSchema(Object enu) { + return ((GenericContainer) enu).getSchema(); + } + + /** Called by the default implementation of {@link #instanceOf}.*/ + protected boolean isMap(Object datum) { + return datum instanceof Map; + } + + /** Called by the default implementation of {@link #instanceOf}.*/ + protected boolean isFixed(Object datum) { + return datum instanceof GenericFixed; + } + + /** Called to obtain the schema of a fixed. By default calls + * {GenericContainer#getSchema(). May be overridden for alternate fixed + * representations. */ + protected Schema getFixedSchema(Object fixed) { + return ((GenericContainer) fixed).getSchema(); + } + + /** Called by the default implementation of {@link #instanceOf}.*/ + protected boolean isString(Object datum) { + return datum instanceof CharSequence; + } + + /** Called by the default implementation of {@link #instanceOf}.*/ + protected boolean isBytes(Object datum) { + return datum instanceof ByteBuffer; + } + + /** + * Called by the default implementation of {@link #instanceOf}. + */ + protected boolean isInteger(Object datum) { + return datum instanceof Integer; + } + + /** + * Called by the default implementation of {@link #instanceOf}. + */ + protected boolean isLong(Object datum) { + return datum instanceof Long; + } + + /** + * Called by the default implementation of {@link #instanceOf}. + */ + protected boolean isFloat(Object datum) { + return datum instanceof Float; + } + + /** + * Called by the default implementation of {@link #instanceOf}. + */ + protected boolean isDouble(Object datum) { + return datum instanceof Double; + } + + /** + * Called by the default implementation of {@link #instanceOf}. + */ + protected boolean isBoolean(Object datum) { + return datum instanceof Boolean; + } + + + /** Compute a hash code according to a schema, consistent with {@link + * #compare(Object, Object, Schema)}. */ + public int hashCode(Object o, Schema s) { + if (o == null) return 0; // incomplete datum + int hashCode = 1; + switch (s.getType()) { + case RECORD: + for (Field f : s.getFields()) { + if (f.order() == Field.Order.IGNORE) + continue; + hashCode = hashCodeAdd(hashCode, + getField(o, f.name(), f.pos()), f.schema()); + } + return hashCode; + case ARRAY: + Collection a = (Collection) o; + Schema elementType = s.getElementType(); + for (Object e : a) + hashCode = hashCodeAdd(hashCode, e, elementType); + return hashCode; + case UNION: + return hashCode(o, s.getTypes().get(resolveUnion(s, o))); + case ENUM: + return s.getEnumOrdinal(o.toString()); + case NULL: + return 0; + case STRING: + return (o instanceof Utf8 ? o : new Utf8(o.toString())).hashCode(); + default: + return o.hashCode(); + } + } + + /** Add the hash code for an object into an accumulated hash code. */ + protected int hashCodeAdd(int hashCode, Object o, Schema s) { + return 31 * hashCode + hashCode(o, s); + } + + /** Compare objects according to their schema. If equal, return zero. If + * greater-than, return 1, if less than return -1. + */ + public int compare(Object o1, Object o2, Schema s) { + return compare(o1, o2, s, false); + } + + /** Comparison implementation. When equals is true, only checks for equality, + * not for order. */ + @SuppressWarnings(value = "unchecked") + protected int compare(Object o1, Object o2, Schema s, boolean equals) { + if (o1 == o2) return 0; + switch (s.getType()) { + case RECORD: + for (Field f : s.getFields()) { + if (f.order() == Field.Order.IGNORE) + continue; // ignore this field + int pos = f.pos(); + String name = f.name(); + int compare = + compare(getField(o1, name, pos), getField(o2, name, pos), + f.schema(), equals); + if (compare != 0) // not equal + return f.order() == Field.Order.DESCENDING ? -compare : compare; + } + return 0; + case ENUM: + return s.getEnumOrdinal(o1.toString()) - s.getEnumOrdinal(o2.toString()); + case ARRAY: + Collection a1 = (Collection) o1; + Collection a2 = (Collection) o2; + Iterator e1 = a1.iterator(); + Iterator e2 = a2.iterator(); + Schema elementType = s.getElementType(); + while (e1.hasNext() && e2.hasNext()) { + int compare = compare(e1.next(), e2.next(), elementType, equals); + if (compare != 0) return compare; + } + return e1.hasNext() ? 1 : (e2.hasNext() ? -1 : 0); + case MAP: + if (equals) + return ((Map) o1).equals(o2) ? 0 : 1; + throw new DataRuntimeException("Can't compare maps!"); + case UNION: + int i1 = resolveUnion(s, o1); + int i2 = resolveUnion(s, o2); + return (i1 == i2) + ? compare(o1, o2, s.getTypes().get(i1), equals) + : i1 - i2; + case NULL: + return 0; + case STRING: + Utf8 u1 = o1 instanceof Utf8 ? (Utf8) o1 : new Utf8(o1.toString()); + Utf8 u2 = o2 instanceof Utf8 ? (Utf8) o2 : new Utf8(o2.toString()); + return u1.compareTo(u2); + default: + return ((Comparable) o1).compareTo(o2); + } + } + + private final Map defaultValueCache + = Collections.synchronizedMap(new WeakHashMap()); + + /** + * Gets the default value of the given field, if any. + * @param field the field whose default value should be retrieved. + * @return the default value associated with the given field, + * or null if none is specified in the schema. + */ + @SuppressWarnings({"rawtypes", "unchecked"}) + public Object getDefaultValue(Field field) { + Object value = field.defaultValue(); + if (value == null) + throw new DataRuntimeException("Field " + field + + " not set and has no default value"); + if (value == null + && (field.schema().getType() == Type.NULL + || (field.schema().getType() == Type.UNION + && field.schema().getTypes().get(0).getType() == Type.NULL))) { + return null; + } + + // Check the cache + Object defaultValue = defaultValueCache.get(field); + + // If not cached, get the default Java value by encoding the default JSON + // value and then decoding it: + if (defaultValue == null) { + defaultValue = field.defaultValue(); + defaultValueCache.put(field, defaultValue); + } + return defaultValue; + } + + private static final Schema STRINGS = Schema.create(Type.STRING); + + /** + * Makes a deep copy of a value given its schema. + * @param schema the schema of the value to deep copy. + * @param value the value to deep copy. + * @return a deep copy of the given value. + */ + @SuppressWarnings({"rawtypes", "unchecked"}) + public T deepCopy(Schema schema, T value) { + if (value == null) { + return null; + } + switch (schema.getType()) { + case ARRAY: + List arrayValue = (List) value; + List arrayCopy = new GenericData.Array( + arrayValue.size(), schema); + for (Object obj : arrayValue) { + arrayCopy.add(deepCopy(schema.getElementType(), obj)); + } + return (T) arrayCopy; + case BOOLEAN: + return value; // immutable + case BYTES: + ByteBuffer byteBufferValue = (ByteBuffer) value; + int start = byteBufferValue.position(); + int length = byteBufferValue.limit() - start; + byte[] bytesCopy = new byte[length]; + byteBufferValue.get(bytesCopy, 0, length); + byteBufferValue.position(start); + return (T) ByteBuffer.wrap(bytesCopy, 0, length); + case DOUBLE: + return value; // immutable + case ENUM: + // Enums are immutable; shallow copy will suffice + return value; + case FIXED: + return (T) createFixed(null, ((GenericFixed) value).bytes(), schema); + case FLOAT: + return value; // immutable + case INT: + return value; // immutable + case LONG: + return value; // immutable + case MAP: + Map mapValue = (Map) value; + Map mapCopy = + new HashMap(mapValue.size()); + for (Map.Entry entry : mapValue.entrySet()) { + mapCopy.put((CharSequence) (deepCopy(STRINGS, entry.getKey())), + deepCopy(schema.getValueType(), entry.getValue())); + } + return (T) mapCopy; + case NULL: + return null; + case RECORD: + Object oldState = getRecordState(value, schema); + Object newRecord = newRecord(null, schema); + Object newState = getRecordState(newRecord, schema); + for (Field f : schema.getFields()) { + int pos = f.pos(); + String name = f.name(); + Object newValue = deepCopy(f.schema(), + getField(value, name, pos, oldState)); + setField(newRecord, name, pos, newValue, newState); + } + return (T) newRecord; + case STRING: + // Strings are immutable + if (value instanceof String) { + return (T) value; + } else if (value instanceof Utf8) { + // Some CharSequence subclasses are mutable, so we still need to make + // a copy + + // Utf8 copy constructor is more efficient than converting + // to string and then back to Utf8 + return (T) new Utf8((Utf8) value); + } + return (T) new Utf8(value.toString()); + case UNION: + return deepCopy(schema.getTypes().get(resolveUnion(schema, value)), value); + default: + throw new DataRuntimeException( + "Deep copy failed for schema \"" + schema + "\" and value \"" + + value + "\""); + } + } + + /** Called to create an fixed value. May be overridden for alternate fixed + * representations. By default, returns {@link GenericFixed}. */ + public Object createFixed(Object old, Schema schema) { + if ((old instanceof GenericFixed) + && ((GenericFixed) old).bytes().length == schema.getFixedSize()) + return old; + return new GenericData.Fixed(schema); + } + + /** Called to create an fixed value. May be overridden for alternate fixed + * representations. By default, returns {@link GenericFixed}. */ + public Object createFixed(Object old, byte[] bytes, Schema schema) { + GenericFixed fixed = (GenericFixed) createFixed(old, schema); + System.arraycopy(bytes, 0, fixed.bytes(), 0, schema.getFixedSize()); + return fixed; + } + + /** Called to create an enum value. May be overridden for alternate enum + * representations. By default, returns a GenericEnumSymbol. */ + public Object createEnum(String symbol, Schema schema) { + return new EnumSymbol(schema, symbol); + } + + /** + * Called to create new record instances. Subclasses may override to use a + * different record implementation. The returned instance must conform to the + * schema provided. If the old object contains fields not present in the + * schema, they should either be removed from the old object, or it should + * create a new instance that conforms to the schema. By default, this returns + * a {@link GenericData.Record}. + */ + public Object newRecord(Object old, Schema schema) { + if (old instanceof IndexedRecord) { + IndexedRecord record = (IndexedRecord) old; + if (record.getSchema() == schema) + return record; + } + return new GenericData.Record(schema); + } + +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericEnumSymbol.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericEnumSymbol.java new file mode 100644 index 00000000000..ebba0776e68 --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericEnumSymbol.java @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +/** An enum symbol. */ +public interface GenericEnumSymbol + extends GenericContainer, Comparable { + /** Return the symbol. */ + String toString(); +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericFixed.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericFixed.java new file mode 100644 index 00000000000..c7a62f36751 --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericFixed.java @@ -0,0 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +/** Fixed-size data. */ +public interface GenericFixed extends GenericContainer { + /** Return the data. */ + byte[] bytes(); +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericRecord.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericRecord.java new file mode 100644 index 00000000000..692e22a8103 --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericRecord.java @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +/** A generic instance of a record schema. Fields are accessible by name as + * well as by index. */ +public interface GenericRecord extends IndexedRecord { + /** Set the value of a field given its name. */ + void put(String key, Object v); + + /** Return the value of a field given its name. */ + Object get(String key); +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericRecordBuilder.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericRecordBuilder.java new file mode 100644 index 00000000000..b80a37857e0 --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/GenericRecordBuilder.java @@ -0,0 +1,259 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +import org.apache.kafka.copycat.data.GenericData.Record; +import org.apache.kafka.copycat.data.Schema.Field; + +import java.io.IOException; + +/** A RecordBuilder for generic records. GenericRecordBuilder fills in default values + * for fields if they are not specified. */ +public class GenericRecordBuilder extends RecordBuilderBase { + private final GenericData.Record record; + + /** + * Creates a GenericRecordBuilder for building Record instances. + * @param schema the schema associated with the record class. + */ + public GenericRecordBuilder(Schema schema) { + super(schema, GenericData.get()); + record = new GenericData.Record(schema); + } + + /** + * Creates a GenericRecordBuilder by copying an existing GenericRecordBuilder. + * @param other the GenericRecordBuilder to copy. + */ + public GenericRecordBuilder(GenericRecordBuilder other) { + super(other, GenericData.get()); + record = new GenericData.Record(other.record, /* deepCopy = */ true); + } + + /** + * Creates a GenericRecordBuilder by copying an existing record instance. + * @param other the record instance to copy. + */ + public GenericRecordBuilder(Record other) { + super(other.getSchema(), GenericData.get()); + record = new GenericData.Record(other, /* deepCopy = */ true); + + // Set all fields in the RecordBuilder that are set in the record + for (Field f : schema().getFields()) { + Object value = other.get(f.pos()); + // Only set the value if it is not null, if the schema type is null, + // or if the schema type is a union that accepts nulls. + if (isValidValue(f, value)) { + set(f, data().deepCopy(f.schema(), value)); + } + } + } + + /** + * Gets the value of a field. + * @param fieldName the name of the field to get. + * @return the value of the field with the given name, or null if not set. + */ + public Object get(String fieldName) { + return get(schema().getField(fieldName)); + } + + /** + * Gets the value of a field. + * @param field the field to get. + * @return the value of the given field, or null if not set. + */ + public Object get(Field field) { + return get(field.pos()); + } + + /** + * Gets the value of a field. + * @param pos the position of the field to get. + * @return the value of the field with the given position, or null if not set. + */ + protected Object get(int pos) { + return record.get(pos); + } + + /** + * Sets the value of a field. + * @param fieldName the name of the field to set. + * @param value the value to set. + * @return a reference to the RecordBuilder. + */ + public GenericRecordBuilder set(String fieldName, Object value) { + return set(schema().getField(fieldName), value); + } + + /** + * Sets the value of a field. + * @param field the field to set. + * @param value the value to set. + * @return a reference to the RecordBuilder. + */ + public GenericRecordBuilder set(Field field, Object value) { + return set(field, field.pos(), value); + } + + /** + * Sets the value of a field. + * @param pos the field to set. + * @param value the value to set. + * @return a reference to the RecordBuilder. + */ + protected GenericRecordBuilder set(int pos, Object value) { + return set(fields()[pos], pos, value); + } + + /** + * Sets the value of a field. + * @param field the field to set. + * @param pos the position of the field. + * @param value the value to set. + * @return a reference to the RecordBuilder. + */ + private GenericRecordBuilder set(Field field, int pos, Object value) { + validate(field, value); + record.put(pos, value); + fieldSetFlags()[pos] = true; + return this; + } + + /** + * Checks whether a field has been set. + * @param fieldName the name of the field to check. + * @return true if the given field is non-null; false otherwise. + */ + public boolean has(String fieldName) { + return has(schema().getField(fieldName)); + } + + /** + * Checks whether a field has been set. + * @param field the field to check. + * @return true if the given field is non-null; false otherwise. + */ + public boolean has(Field field) { + return has(field.pos()); + } + + /** + * Checks whether a field has been set. + * @param pos the position of the field to check. + * @return true if the given field is non-null; false otherwise. + */ + protected boolean has(int pos) { + return fieldSetFlags()[pos]; + } + + /** + * Clears the value of the given field. + * @param fieldName the name of the field to clear. + * @return a reference to the RecordBuilder. + */ + public GenericRecordBuilder clear(String fieldName) { + return clear(schema().getField(fieldName)); + } + + /** + * Clears the value of the given field. + * @param field the field to clear. + * @return a reference to the RecordBuilder. + */ + public GenericRecordBuilder clear(Field field) { + return clear(field.pos()); + } + + /** + * Clears the value of the given field. + * @param pos the position of the field to clear. + * @return a reference to the RecordBuilder. + */ + protected GenericRecordBuilder clear(int pos) { + record.put(pos, null); + fieldSetFlags()[pos] = false; + return this; + } + + @Override + public Record build() { + Record record; + try { + record = new GenericData.Record(schema()); + } catch (Exception e) { + throw new DataRuntimeException(e); + } + + for (Field field : fields()) { + Object value; + try { + value = getWithDefault(field); + } catch (IOException e) { + throw new DataRuntimeException(e); + } + if (value != null) { + record.put(field.pos(), value); + } + } + + return record; + } + + /** + * Gets the value of the given field. + * If the field has been set, the set value is returned (even if it's null). + * If the field hasn't been set and has a default value, the default value + * is returned. + * @param field the field whose value should be retrieved. + * @return the value set for the given field, the field's default value, + * or null. + * @throws IOException + */ + private Object getWithDefault(Field field) throws IOException { + return fieldSetFlags()[field.pos()] ? + record.get(field.pos()) : defaultValue(field); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = super.hashCode(); + result = prime * result + ((record == null) ? 0 : record.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (!super.equals(obj)) + return false; + if (getClass() != obj.getClass()) + return false; + GenericRecordBuilder other = (GenericRecordBuilder) obj; + if (record == null) { + if (other.record != null) + return false; + } else if (!record.equals(other.record)) + return false; + return true; + } +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/IndexedRecord.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/IndexedRecord.java new file mode 100644 index 00000000000..b8deb4b639d --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/IndexedRecord.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +/** A record implementation that permits field access by integer index.*/ +public interface IndexedRecord extends GenericContainer { + /** Set the value of a field given its position in the schema. + *

This method is not meant to be called by user code, but only internally for deep copying */ + void put(int i, Object v); + + /** Return the value of a field given its position in the schema. + *

This method is not meant to be called by user code, but only internally for deep copying */ + Object get(int i); +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/ObjectProperties.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/ObjectProperties.java new file mode 100644 index 00000000000..e995b7f89c0 --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/ObjectProperties.java @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; + +/** + * Base class for objects that have Object-valued properties. + */ +public abstract class ObjectProperties { + public static class Null { + private Null() { + } + } + + /** A value representing a JSON null. */ + public static final Null NULL_VALUE = new Null(); + + Map props = new LinkedHashMap(1); + + private Set reserved; + + ObjectProperties(Set reserved) { + this.reserved = reserved; + } + + /** + * Returns the value of the named, string-valued property in this schema. + * Returns null if there is no string-valued property with that name. + */ + public String getProp(String name) { + Object value = getObjectProp(name); + return (value instanceof String) ? (String) value : null; + } + + /** + * Returns the value of the named property in this schema. + * Returns null if there is no property with that name. + */ + public synchronized Object getObjectProp(String name) { + return props.get(name); + } + + /** + * Adds a property with the given name name and + * value value. Neither name nor value can be + * null. It is illegal to add a property if another with + * the same name but different value already exists in this schema. + * + * @param name The name of the property to add + * @param value The value for the property to add + */ + public synchronized void addProp(String name, Object value) { + if (reserved.contains(name)) + throw new DataRuntimeException("Can't set reserved property: " + name); + + if (value == null) + throw new DataRuntimeException("Can't set a property to null: " + name); + + Object old = props.get(name); + if (old == null) + props.put(name, value); + else if (!old.equals(value)) + throw new DataRuntimeException("Can't overwrite property: " + name); + } +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/RecordBuilder.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/RecordBuilder.java new file mode 100644 index 00000000000..2fef55d9cb9 --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/RecordBuilder.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + + +/** Interface for record builders */ +public interface RecordBuilder { + /** + * Constructs a new instance using the values set in the RecordBuilder. + * If a particular value was not set and the schema defines a default + * value, the default value will be used. + * @return a new instance using values set in the RecordBuilder. + */ + T build(); +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/RecordBuilderBase.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/RecordBuilderBase.java new file mode 100644 index 00000000000..41f22550885 --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/RecordBuilderBase.java @@ -0,0 +1,173 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +import org.apache.kafka.copycat.data.Schema.Field; +import org.apache.kafka.copycat.data.Schema.Type; + +import java.io.IOException; +import java.util.Arrays; + +/** Abstract base class for RecordBuilder implementations. Not thread-safe. */ +public abstract class RecordBuilderBase + implements RecordBuilder { + private static final Field[] EMPTY_FIELDS = new Field[0]; + private final Schema schema; + private final Field[] fields; + private final boolean[] fieldSetFlags; + private final GenericData data; + + protected final Schema schema() { + return schema; + } + + protected final Field[] fields() { + return fields; + } + + protected final boolean[] fieldSetFlags() { + return fieldSetFlags; + } + + protected final GenericData data() { + return data; + } + + /** + * Creates a RecordBuilderBase for building records of the given type. + * @param schema the schema associated with the record class. + */ + protected RecordBuilderBase(Schema schema, GenericData data) { + this.schema = schema; + this.data = data; + fields = (Field[]) schema.getFields().toArray(EMPTY_FIELDS); + fieldSetFlags = new boolean[fields.length]; + } + + /** + * RecordBuilderBase copy constructor. + * Makes a deep copy of the values in the other builder. + * @param other RecordBuilderBase instance to copy. + */ + protected RecordBuilderBase(RecordBuilderBase other, GenericData data) { + this.schema = other.schema; + this.data = data; + fields = (Field[]) schema.getFields().toArray(EMPTY_FIELDS); + fieldSetFlags = new boolean[other.fieldSetFlags.length]; + System.arraycopy( + other.fieldSetFlags, 0, fieldSetFlags, 0, fieldSetFlags.length); + } + + /** + * Validates that a particular value for a given field is valid according to + * the following algorithm: + * 1. If the value is not null, or the field type is null, or the field type + * is a union which accepts nulls, returns. + * 2. Else, if the field has a default value, returns. + * 3. Otherwise throws AvroRuntimeException. + * @param field the field to validate. + * @param value the value to validate. + * @throws NullPointerException if value is null and the given field does + * not accept null values. + */ + protected void validate(Field field, Object value) { + if (isValidValue(field, value)) { + return; + } else if (field.defaultValue() != null) { + return; + } else { + throw new DataRuntimeException( + "Field " + field + " does not accept null values"); + } + } + + /** + * Tests whether a value is valid for a specified field. + * @param f the field for which to test the value. + * @param value the value to test. + * @return true if the value is valid for the given field; false otherwise. + */ + protected static boolean isValidValue(Field f, Object value) { + if (value != null) { + return true; + } + + Schema schema = f.schema(); + Type type = schema.getType(); + + // If the type is null, any value is valid + if (type == Type.NULL) { + return true; + } + + // If the type is a union that allows nulls, any value is valid + if (type == Type.UNION) { + for (Schema s : schema.getTypes()) { + if (s.getType() == Type.NULL) { + return true; + } + } + } + + // The value is null but the type does not allow nulls + return false; + } + + /** + * Gets the default value of the given field, if any. + * @param field the field whose default value should be retrieved. + * @return the default value associated with the given field, + * or null if none is specified in the schema. + * @throws IOException + */ + @SuppressWarnings({"rawtypes", "unchecked"}) + protected Object defaultValue(Field field) throws IOException { + return data.deepCopy(field.schema(), data.getDefaultValue(field)); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + Arrays.hashCode(fieldSetFlags); + result = prime * result + ((schema == null) ? 0 : schema.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + @SuppressWarnings("rawtypes") + RecordBuilderBase other = (RecordBuilderBase) obj; + if (!Arrays.equals(fieldSetFlags, other.fieldSetFlags)) + return false; + if (schema == null) { + if (other.schema != null) + return false; + } else if (!schema.equals(other.schema)) + return false; + return true; + } +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/Schema.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/Schema.java new file mode 100644 index 00000000000..8976ba034b0 --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/Schema.java @@ -0,0 +1,1210 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +import java.nio.ByteBuffer; +import java.util.*; + +/** An abstract data type. + *

A schema may be one of: + *

    + *
  • A record, mapping field names to field value data; + *
  • An enum, containing one of a small set of symbols; + *
  • An array of values, all of the same schema; + *
  • A map, containing string/value pairs, of a declared schema; + *
  • A union of other schemas; + *
  • A fixed sized binary object; + *
  • A unicode string; + *
  • A sequence of bytes; + *
  • A 32-bit signed int; + *
  • A 64-bit signed long; + *
  • A 32-bit IEEE single-float; or + *
  • A 64-bit IEEE double-float; or + *
  • A boolean; or + *
  • null. + *
+ * + * A schema can be constructed using one of its static createXXX + * methods, or more conveniently using {@link SchemaBuilder}. The schema objects are + * logically immutable. + * There are only two mutating methods - {@link #setFields(List)} and + * {@link #addProp(String, Object)}. The following restrictions apply on these + * two methods. + *
    + *
  • {@link #setFields(List)}, can be called at most once. This method exists + * in order to enable clients to build recursive schemas. + *
  • {@link #addProp(String, Object)} can be called with property names + * that are not present already. It is not possible to change or delete an + * existing property. + *
+ */ +public abstract class Schema extends ObjectProperties { + private static final int NO_HASHCODE = Integer.MIN_VALUE; + + /** The type of a schema. */ + public enum Type { + RECORD { + @Override + public Object defaultValue(Schema schema) { + return new GenericRecordBuilder(schema).build(); + } + }, + ENUM { + @Override + public Object defaultValue(Schema schema) { + return null; + } + }, + ARRAY { + @Override + public Object defaultValue(Schema schema) { + return new GenericData.Array(0, schema); + } + }, + MAP { + @Override + public Object defaultValue(Schema schema) { + return new HashMap(); + } + }, + UNION { + @Override + public Object defaultValue(Schema schema) { + Schema firstSchema = schema.getTypes().get(0); + return firstSchema.getType().defaultValue(firstSchema); + } + }, + FIXED { + @Override + public Object defaultValue(Schema schema) { + return new GenericData.Fixed(schema); + } + }, + STRING { + @Override + public Object defaultValue(Schema schema) { + return ""; + } + }, + BYTES { + @Override + public Object defaultValue(Schema schema) { + return new byte[0]; + } + }, + INT { + @Override + public Object defaultValue(Schema schema) { + return 0; + } + }, + LONG { + @Override + public Object defaultValue(Schema schema) { + return 0; + } + }, + FLOAT { + @Override + public Object defaultValue(Schema schema) { + return 0; + } + }, + DOUBLE { + @Override + public Object defaultValue(Schema schema) { + return 0; + } + }, + BOOLEAN { + @Override + public Object defaultValue(Schema schema) { + return false; + } + }, + NULL { + @Override + public Object defaultValue(Schema schema) { + return null; + } + }; + private String name; + + private Type() { + this.name = this.name().toLowerCase(); + } + + public String getName() { + return name; + } + + public abstract Object defaultValue(Schema schema); + } + + private final Type type; + + Schema(Type type) { + super(SCHEMA_RESERVED); + this.type = type; + } + + /** Create a schema for a primitive type. */ + public static Schema create(Type type) { + switch (type) { + case STRING: + return new StringSchema(); + case BYTES: + return new BytesSchema(); + case INT: + return new IntSchema(); + case LONG: + return new LongSchema(); + case FLOAT: + return new FloatSchema(); + case DOUBLE: + return new DoubleSchema(); + case BOOLEAN: + return new BooleanSchema(); + case NULL: + return new NullSchema(); + default: + throw new DataRuntimeException("Can't create a: " + type); + } + } + + private static final Set SCHEMA_RESERVED = new HashSet(); + + static { + Collections.addAll(SCHEMA_RESERVED, + "doc", "fields", "items", "name", "namespace", + "size", "symbols", "values", "type", "aliases"); + } + + int hashCode = NO_HASHCODE; + + @Override + public void addProp(String name, Object value) { + super.addProp(name, value); + hashCode = NO_HASHCODE; + } + + /** Create an anonymous record schema. */ + public static Schema createRecord(List fields) { + Schema result = createRecord(null, null, null, false); + result.setFields(fields); + return result; + } + + /** Create a named record schema. */ + public static Schema createRecord(String name, String doc, String namespace, + boolean isError) { + return new RecordSchema(new Name(name, namespace), doc, isError); + } + + /** Create an enum schema. */ + public static Schema createEnum(String name, String doc, String namespace, + List values) { + return new EnumSchema(new Name(name, namespace), doc, + new LockableArrayList(values)); + } + + /** Create an array schema. */ + public static Schema createArray(Schema elementType) { + return new ArraySchema(elementType); + } + + /** Create a map schema. */ + public static Schema createMap(Schema valueType) { + return new MapSchema(valueType); + } + + /** Create a union schema. */ + public static Schema createUnion(List types) { + return new UnionSchema(new LockableArrayList(types)); + } + + /** Create a union schema. */ + public static Schema createUnion(Schema... types) { + return createUnion(new LockableArrayList(types)); + } + + /** Create a union schema. */ + public static Schema createFixed(String name, String doc, String space, + int size) { + return new FixedSchema(new Name(name, space), doc, size); + } + + /** Return the type of this schema. */ + public Type getType() { + return type; + } + + /** + * If this is a record, returns the Field with the + * given name fieldName. If there is no field by that name, a + * null is returned. + */ + public Field getField(String fieldname) { + throw new DataRuntimeException("Not a record: " + this); + } + + /** + * If this is a record, returns the fields in it. The returned + * list is in the order of their positions. + */ + public List getFields() { + throw new DataRuntimeException("Not a record: " + this); + } + + /** + * If this is a record, set its fields. The fields can be set + * only once in a schema. + */ + public void setFields(List fields) { + throw new DataRuntimeException("Not a record: " + this); + } + + /** If this is an enum, return its symbols. */ + public List getEnumSymbols() { + throw new DataRuntimeException("Not an enum: " + this); + } + + /** If this is an enum, return a symbol's ordinal value. */ + public int getEnumOrdinal(String symbol) { + throw new DataRuntimeException("Not an enum: " + this); + } + + /** If this is an enum, returns true if it contains given symbol. */ + public boolean hasEnumSymbol(String symbol) { + throw new DataRuntimeException("Not an enum: " + this); + } + + /** If this is a record, enum or fixed, returns its name, otherwise the name + * of the primitive type. */ + public String getName() { + return type.name; + } + + /** If this is a record, enum, or fixed, returns its docstring, + * if available. Otherwise, returns null. */ + public String getDoc() { + return null; + } + + /** If this is a record, enum or fixed, returns its namespace, if any. */ + public String getNamespace() { + throw new DataRuntimeException("Not a named type: " + this); + } + + /** If this is a record, enum or fixed, returns its namespace-qualified name, + * otherwise returns the name of the primitive type. */ + public String getFullName() { + return getName(); + } + + /** If this is a record, enum or fixed, add an alias. */ + public void addAlias(String alias) { + throw new DataRuntimeException("Not a named type: " + this); + } + + /** If this is a record, enum or fixed, add an alias. */ + public void addAlias(String alias, String space) { + throw new DataRuntimeException("Not a named type: " + this); + } + + /** If this is a record, enum or fixed, return its aliases, if any. */ + public Set getAliases() { + throw new DataRuntimeException("Not a named type: " + this); + } + + /** Returns true if this record is an error type. */ + public boolean isError() { + throw new DataRuntimeException("Not a record: " + this); + } + + /** If this is an array, returns its element type. */ + public Schema getElementType() { + throw new DataRuntimeException("Not an array: " + this); + } + + /** If this is a map, returns its value type. */ + public Schema getValueType() { + throw new DataRuntimeException("Not a map: " + this); + } + + /** If this is a union, returns its types. */ + public List getTypes() { + throw new DataRuntimeException("Not a union: " + this); + } + + /** If this is a union, return the branch with the provided full name. */ + public Integer getIndexNamed(String name) { + throw new DataRuntimeException("Not a union: " + this); + } + + /** If this is fixed, returns its size. */ + public int getFixedSize() { + throw new DataRuntimeException("Not fixed: " + this); + } + + @Override + public String toString() { + // FIXME A more JSON-like output showing the details would be nice + return "Schema:" + this.getType() + ":" + getFullName(); + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof Schema)) return false; + Schema that = (Schema) o; + if (!(this.type == that.type)) return false; + return equalCachedHash(that) && props.equals(that.props); + } + + public final int hashCode() { + if (hashCode == NO_HASHCODE) + hashCode = computeHash(); + return hashCode; + } + + int computeHash() { + return getType().hashCode() + props.hashCode(); + } + + final boolean equalCachedHash(Schema other) { + return (hashCode == other.hashCode) + || (hashCode == NO_HASHCODE) + || (other.hashCode == NO_HASHCODE); + } + + private static final Set FIELD_RESERVED = new HashSet(); + + static { + Collections.addAll(FIELD_RESERVED, + "default", "doc", "name", "order", "type", "aliases"); + } + + /** A field within a record. */ + public static class Field extends ObjectProperties { + + /** How values of this field should be ordered when sorting records. */ + public enum Order { + ASCENDING, DESCENDING, IGNORE; + private String name; + + private Order() { + this.name = this.name().toLowerCase(); + } + } + + + private final String name; // name of the field. + private int position = -1; + private final Schema schema; + private final String doc; + private final Object defaultValue; + private final Order order; + private Set aliases; + + public Field(String name, Schema schema, String doc, + Object defaultValue) { + this(name, schema, doc, defaultValue, Order.ASCENDING); + } + + public Field(String name, Schema schema, String doc, + Object defaultValue, Order order) { + super(FIELD_RESERVED); + this.name = validateName(name); + this.schema = schema; + this.doc = doc; + this.defaultValue = validateDefault(name, schema, defaultValue); + this.order = order; + } + + public String name() { + return name; + } + + + /** The position of this field within the record. */ + public int pos() { + return position; + } + + /** This field's {@link Schema}. */ + public Schema schema() { + return schema; + } + + /** Field's documentation within the record, if set. May return null. */ + public String doc() { + return doc; + } + + public Object defaultValue() { + return defaultValue; + } + + public Order order() { + return order; + } + + public void addAlias(String alias) { + if (aliases == null) + this.aliases = new LinkedHashSet(); + aliases.add(alias); + } + + /** Return the defined aliases as an unmodifieable Set. */ + public Set aliases() { + if (aliases == null) + return Collections.emptySet(); + return Collections.unmodifiableSet(aliases); + } + + public boolean equals(Object other) { + if (other == this) return true; + if (!(other instanceof Field)) return false; + Field that = (Field) other; + return (name.equals(that.name)) && + (schema.equals(that.schema)) && + defaultValueEquals(that.defaultValue) && + (order == that.order) && + props.equals(that.props); + } + + public int hashCode() { + return name.hashCode() + schema.computeHash(); + } + + /** Do any possible implicit conversions to double, or return 0 if there isn't a + * valid conversion */ + private double doubleValue(Object v) { + if (v instanceof Integer) + return (double) (Integer) v; + else if (v instanceof Long) + return (double) (Long) v; + else if (v instanceof Float) + return (double) (Float) v; + else if (v instanceof Double) + return (double) (Double) v; + else + return 0; + } + + private boolean defaultValueEquals(Object thatDefaultValue) { + if (defaultValue == null) + return thatDefaultValue == null; + if (Double.isNaN(doubleValue(defaultValue))) + return Double.isNaN(doubleValue(thatDefaultValue)); + return defaultValue.equals(thatDefaultValue); + } + + @Override + public String toString() { + return name + " type:" + schema.type + " pos:" + position; + } + } + + static class Name { + private final String name; + private final String space; + private final String full; + + public Name(String name, String space) { + if (name == null) { // anonymous + this.name = this.space = this.full = null; + return; + } + int lastDot = name.lastIndexOf('.'); + if (lastDot < 0) { // unqualified name + this.name = validateName(name); + } else { // qualified name + space = name.substring(0, lastDot); // get space from name + this.name = validateName(name.substring(lastDot + 1, name.length())); + } + if ("".equals(space)) + space = null; + this.space = space; + this.full = (this.space == null) ? this.name : this.space + "." + this.name; + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof Name)) return false; + Name that = (Name) o; + return full == null ? that.full == null : full.equals(that.full); + } + + public int hashCode() { + return full == null ? 0 : full.hashCode(); + } + + public String toString() { + return full; + } + + public String getQualified(String defaultSpace) { + return (space == null || space.equals(defaultSpace)) ? name : full; + } + } + + private static abstract class NamedSchema extends Schema { + final Name name; + final String doc; + Set aliases; + + public NamedSchema(Type type, Name name, String doc) { + super(type); + this.name = name; + this.doc = doc; + if (PRIMITIVES.containsKey(name.full)) { + throw new DataTypeException("Schemas may not be named after primitives: " + name.full); + } + } + + public String getName() { + return name.name; + } + + public String getDoc() { + return doc; + } + + public String getNamespace() { + return name.space; + } + + public String getFullName() { + return name.full; + } + + public void addAlias(String alias) { + addAlias(alias, null); + } + + public void addAlias(String name, String space) { + if (aliases == null) + this.aliases = new LinkedHashSet(); + if (space == null) + space = this.name.space; + aliases.add(new Name(name, space)); + } + + public Set getAliases() { + Set result = new LinkedHashSet(); + if (aliases != null) + for (Name alias : aliases) + result.add(alias.full); + return result; + } + + public boolean equalNames(NamedSchema that) { + return this.name.equals(that.name); + } + + @Override + int computeHash() { + return super.computeHash() + name.hashCode(); + } + } + + private static class SeenPair { + private Object s1; + private Object s2; + + private SeenPair(Object s1, Object s2) { + this.s1 = s1; + this.s2 = s2; + } + + public boolean equals(Object o) { + return this.s1 == ((SeenPair) o).s1 && this.s2 == ((SeenPair) o).s2; + } + + public int hashCode() { + return System.identityHashCode(s1) + System.identityHashCode(s2); + } + } + + private static final ThreadLocal SEEN_EQUALS = new ThreadLocal() { + protected Set initialValue() { + return new HashSet(); + } + }; + private static final ThreadLocal SEEN_HASHCODE = new ThreadLocal() { + protected Map initialValue() { + return new IdentityHashMap(); + } + }; + + @SuppressWarnings(value = "unchecked") + private static class RecordSchema extends NamedSchema { + private List fields; + private Map fieldMap; + private final boolean isError; + + public RecordSchema(Name name, String doc, boolean isError) { + super(Type.RECORD, name, doc); + this.isError = isError; + } + + public boolean isError() { + return isError; + } + + @Override + public Field getField(String fieldname) { + if (fieldMap == null) + throw new DataRuntimeException("Schema fields not set yet"); + return fieldMap.get(fieldname); + } + + @Override + public List getFields() { + if (fields == null) + throw new DataRuntimeException("Schema fields not set yet"); + return fields; + } + + @Override + public void setFields(List fields) { + if (this.fields != null) { + throw new DataRuntimeException("Fields are already set"); + } + int i = 0; + fieldMap = new HashMap(); + LockableArrayList ff = new LockableArrayList(); + for (Field f : fields) { + if (f.position != -1) + throw new DataRuntimeException("Field already used: " + f); + f.position = i++; + final Field existingField = fieldMap.put(f.name(), f); + if (existingField != null) { + throw new DataRuntimeException(String.format( + "Duplicate field %s in record %s: %s and %s.", + f.name(), name, f, existingField)); + } + ff.add(f); + } + this.fields = ff.lock(); + this.hashCode = NO_HASHCODE; + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof RecordSchema)) return false; + RecordSchema that = (RecordSchema) o; + if (!equalCachedHash(that)) return false; + if (!equalNames(that)) return false; + if (!props.equals(that.props)) return false; + Set seen = SEEN_EQUALS.get(); + SeenPair here = new SeenPair(this, o); + if (seen.contains(here)) return true; // prevent stack overflow + boolean first = seen.isEmpty(); + try { + seen.add(here); + return fields.equals(((RecordSchema) o).fields); + } finally { + if (first) seen.clear(); + } + } + + @Override + int computeHash() { + Map seen = SEEN_HASHCODE.get(); + if (seen.containsKey(this)) return 0; // prevent stack overflow + boolean first = seen.isEmpty(); + try { + seen.put(this, this); + return super.computeHash() + fields.hashCode(); + } finally { + if (first) seen.clear(); + } + } + } + + private static class EnumSchema extends NamedSchema { + private final List symbols; + private final Map ordinals; + + public EnumSchema(Name name, String doc, + LockableArrayList symbols) { + super(Type.ENUM, name, doc); + this.symbols = symbols.lock(); + this.ordinals = new HashMap(); + int i = 0; + for (String symbol : symbols) + if (ordinals.put(validateName(symbol), i++) != null) + throw new SchemaParseException("Duplicate enum symbol: " + symbol); + } + + public List getEnumSymbols() { + return symbols; + } + + public boolean hasEnumSymbol(String symbol) { + return ordinals.containsKey(symbol); + } + + public int getEnumOrdinal(String symbol) { + return ordinals.get(symbol); + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof EnumSchema)) return false; + EnumSchema that = (EnumSchema) o; + return equalCachedHash(that) + && equalNames(that) + && symbols.equals(that.symbols) + && props.equals(that.props); + } + + @Override + int computeHash() { + return super.computeHash() + symbols.hashCode(); + } + } + + private static class ArraySchema extends Schema { + private final Schema elementType; + + public ArraySchema(Schema elementType) { + super(Type.ARRAY); + this.elementType = elementType; + } + + public Schema getElementType() { + return elementType; + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof ArraySchema)) return false; + ArraySchema that = (ArraySchema) o; + return equalCachedHash(that) + && elementType.equals(that.elementType) + && props.equals(that.props); + } + + @Override + int computeHash() { + return super.computeHash() + elementType.computeHash(); + } + } + + private static class MapSchema extends Schema { + private final Schema valueType; + + public MapSchema(Schema valueType) { + super(Type.MAP); + this.valueType = valueType; + } + + public Schema getValueType() { + return valueType; + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof MapSchema)) return false; + MapSchema that = (MapSchema) o; + return equalCachedHash(that) + && valueType.equals(that.valueType) + && props.equals(that.props); + } + + @Override + int computeHash() { + return super.computeHash() + valueType.computeHash(); + } + } + + private static class UnionSchema extends Schema { + private final List types; + private final Map indexByName + = new HashMap(); + + public UnionSchema(LockableArrayList types) { + super(Type.UNION); + this.types = types.lock(); + int index = 0; + for (Schema type : types) { + if (type.getType() == Type.UNION) + throw new DataRuntimeException("Nested union: " + this); + String name = type.getFullName(); + if (name == null) + throw new DataRuntimeException("Nameless in union:" + this); + if (indexByName.put(name, index++) != null) + throw new DataRuntimeException("Duplicate in union:" + name); + } + } + + public List getTypes() { + return types; + } + + public Integer getIndexNamed(String name) { + return indexByName.get(name); + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof UnionSchema)) return false; + UnionSchema that = (UnionSchema) o; + return equalCachedHash(that) + && types.equals(that.types) + && props.equals(that.props); + } + + @Override + int computeHash() { + int hash = super.computeHash(); + for (Schema type : types) + hash += type.computeHash(); + return hash; + } + } + + private static class FixedSchema extends NamedSchema { + private final int size; + + public FixedSchema(Name name, String doc, int size) { + super(Type.FIXED, name, doc); + if (size < 0) + throw new IllegalArgumentException("Invalid fixed size: " + size); + this.size = size; + } + + public int getFixedSize() { + return size; + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof FixedSchema)) return false; + FixedSchema that = (FixedSchema) o; + return equalCachedHash(that) + && equalNames(that) + && size == that.size + && props.equals(that.props); + } + + @Override + int computeHash() { + return super.computeHash() + size; + } + } + + private static class StringSchema extends Schema { + public StringSchema() { + super(Type.STRING); + } + } + + private static class BytesSchema extends Schema { + public BytesSchema() { + super(Type.BYTES); + } + } + + private static class IntSchema extends Schema { + public IntSchema() { + super(Type.INT); + } + } + + private static class LongSchema extends Schema { + public LongSchema() { + super(Type.LONG); + } + } + + private static class FloatSchema extends Schema { + public FloatSchema() { + super(Type.FLOAT); + } + } + + private static class DoubleSchema extends Schema { + public DoubleSchema() { + super(Type.DOUBLE); + } + } + + private static class BooleanSchema extends Schema { + public BooleanSchema() { + super(Type.BOOLEAN); + } + } + + private static class NullSchema extends Schema { + public NullSchema() { + super(Type.NULL); + } + } + + static final Map PRIMITIVES = new HashMap(); + + static { + PRIMITIVES.put("string", Type.STRING); + PRIMITIVES.put("bytes", Type.BYTES); + PRIMITIVES.put("int", Type.INT); + PRIMITIVES.put("long", Type.LONG); + PRIMITIVES.put("float", Type.FLOAT); + PRIMITIVES.put("double", Type.DOUBLE); + PRIMITIVES.put("boolean", Type.BOOLEAN); + PRIMITIVES.put("null", Type.NULL); + } + + static class Names extends LinkedHashMap { + private String space; // default namespace + + public Names() { + } + + public Names(String space) { + this.space = space; + } + + public String space() { + return space; + } + + public void space(String space) { + this.space = space; + } + + @Override + public Schema get(Object o) { + Name name; + if (o instanceof String) { + Type primitive = PRIMITIVES.get((String) o); + if (primitive != null) return Schema.create(primitive); + name = new Name((String) o, space); + if (!containsKey(name)) // if not in default + name = new Name((String) o, ""); // try anonymous + } else { + name = (Name) o; + } + return super.get(name); + } + + public boolean contains(Schema schema) { + return get(((NamedSchema) schema).name) != null; + } + + public void add(Schema schema) { + put(((NamedSchema) schema).name, schema); + } + + @Override + public Schema put(Name name, Schema schema) { + if (containsKey(name)) + throw new SchemaParseException("Can't redefine: " + name); + return super.put(name, schema); + } + } + + private static ThreadLocal validateNames + = new ThreadLocal() { + @Override + protected Boolean initialValue() { + return true; + } + }; + + private static String validateName(String name) { + if (!validateNames.get()) return name; // not validating names + int length = name.length(); + if (length == 0) + throw new SchemaParseException("Empty name"); + char first = name.charAt(0); + if (!(Character.isLetter(first) || first == '_')) + throw new SchemaParseException("Illegal initial character: " + name); + for (int i = 1; i < length; i++) { + char c = name.charAt(i); + if (!(Character.isLetterOrDigit(c) || c == '_')) + throw new SchemaParseException("Illegal character in: " + name); + } + return name; + } + + private static final ThreadLocal VALIDATE_DEFAULTS + = new ThreadLocal() { + @Override + protected Boolean initialValue() { + return false; + } + }; + + private static Object validateDefault(String fieldName, Schema schema, + Object defaultValue) { + if ((defaultValue != null) + && !isValidDefault(schema, defaultValue)) { // invalid default + String message = "Invalid default for field " + fieldName + + ": " + defaultValue + " not a " + schema; + if (VALIDATE_DEFAULTS.get()) + throw new DataTypeException(message); // throw exception + System.err.println("[WARNING] Avro: " + message); // or log warning + } + return defaultValue; + } + + private static boolean isValidDefault(Schema schema, Object defaultValue) { + switch (schema.getType()) { + case STRING: + case ENUM: + return (defaultValue instanceof String); + case BYTES: + case FIXED: + return (defaultValue instanceof byte[] || defaultValue instanceof ByteBuffer); + case INT: + return (defaultValue instanceof Integer); + case LONG: + return (defaultValue instanceof Long); + case FLOAT: + return (defaultValue instanceof Float); + case DOUBLE: + return (defaultValue instanceof Double); + case BOOLEAN: + return (defaultValue instanceof Boolean); + case NULL: + return defaultValue == null; + case ARRAY: + if (!(defaultValue instanceof Collection)) + return false; + for (Object element : (Collection) defaultValue) + if (!isValidDefault(schema.getElementType(), element)) + return false; + return true; + case MAP: + if (!(defaultValue instanceof Map)) + return false; + for (Object value : ((Map) defaultValue).values()) + if (!isValidDefault(schema.getValueType(), value)) + return false; + return true; + case UNION: // union default: first branch + return isValidDefault(schema.getTypes().get(0), defaultValue); + case RECORD: + if (!(defaultValue instanceof GenericData.Record)) + return false; + for (Field field : schema.getFields()) + if (!isValidDefault(field.schema(), ((GenericData.Record) defaultValue).get(field.name()))) + return false; + return true; + default: + return false; + } + } + + /** + * No change is permitted on LockableArrayList once lock() has been + * called on it. + * @param + */ + + /* + * This class keeps a boolean variable locked which is set + * to true in the lock() method. It's legal to call + * lock() any number of times. Any lock() other than the first one + * is a no-op. + * + * This class throws IllegalStateException if a mutating + * operation is performed after being locked. Since modifications through + * iterator also use the list's mutating operations, this effectively + * blocks all modifications. + */ + static class LockableArrayList extends ArrayList { + private static final long serialVersionUID = 1L; + private boolean locked = false; + + public LockableArrayList() { + } + + public LockableArrayList(int size) { + super(size); + } + + public LockableArrayList(List types) { + super(types); + } + + public LockableArrayList(E... types) { + super(types.length); + Collections.addAll(this, types); + } + + public List lock() { + locked = true; + return this; + } + + private void ensureUnlocked() { + if (locked) { + throw new IllegalStateException(); + } + } + + public boolean add(E e) { + ensureUnlocked(); + return super.add(e); + } + + public boolean remove(Object o) { + ensureUnlocked(); + return super.remove(o); + } + + public E remove(int index) { + ensureUnlocked(); + return super.remove(index); + } + + public boolean addAll(Collection c) { + ensureUnlocked(); + return super.addAll(c); + } + + public boolean addAll(int index, Collection c) { + ensureUnlocked(); + return super.addAll(index, c); + } + + public boolean removeAll(Collection c) { + ensureUnlocked(); + return super.removeAll(c); + } + + public boolean retainAll(Collection c) { + ensureUnlocked(); + return super.retainAll(c); + } + + public void clear() { + ensureUnlocked(); + super.clear(); + } + + } + +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/SchemaBuilder.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/SchemaBuilder.java new file mode 100644 index 00000000000..d68383ea2ae --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/SchemaBuilder.java @@ -0,0 +1,2560 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +import org.apache.kafka.copycat.data.Schema.Field; + +import java.nio.ByteBuffer; +import java.util.*; + +/** + *

+ * A fluent interface for building {@link Schema} instances. The flow of the API + * is designed to mimic the Avro Schema + * Specification + *

+ * For example, the below JSON schema and the fluent builder code to create it + * are very similar: + * + *
+ * {
+ *   "type": "record",
+ *   "name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
+ *   "fields": [
+ *     {"name": "clientHash",
+ *      "type": {"type": "fixed", "name": "MD5", "size": 16}},
+ *     {"name": "clientProtocol", "type": ["null", "string"]},
+ *     {"name": "serverHash", "type": "MD5"},
+ *     {"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
+ *   ]
+ * }
+ * 
+ * + *
+ *   Schema schema = SchemaBuilder
+ *   .record("HandshakeRequest").namespace("org.apache.avro.ipc)
+ *   .fields()
+ *     .name("clientHash").type().fixed("MD5").size(16).noDefault()
+ *     .name("clientProtocol").type().nullable().stringType().noDefault()
+ *     .name("serverHash").type("MD5")
+ *     .name("meta").type().nullable().map().values().bytesType().noDefault()
+ *   .endRecord();
+ * 
+ *

+ * + *

Usage Guide
+ * SchemaBuilder chains together many smaller builders and maintains nested + * context in order to mimic the Avro Schema specification. Every Avro type in + * JSON has required and optional JSON properties, as well as user-defined + * properties. + *

+ *

Selecting and Building an Avro Type
+ * The API analogy for the right hand side of the Avro Schema JSON + *
+ * "type":
+ * 
+ * is a {@link TypeBuilder}, {@link FieldTypeBuilder}, or + * {@link UnionFieldTypeBuilder}, depending on the context. These types all + * share a similar API for selecting and building types. + *

+ *

Primitive Types
+ * All Avro primitive types are trivial to configure. A primitive type in + * Avro JSON can be declared two ways, one that supports custom properties + * and one that does not: + *
+ * {"type":"int"}
+ * {"type":{"name":"int"}}
+ * {"type":{"name":"int", "customProp":"val"}}
+ * 
+ * The analogous code form for the above three JSON lines are the below + * three lines: + *
+ *  .intType()
+ *  .intBuilder().endInt()
+ *  .intBuilder().prop("customProp", "val").endInt()
+ * 
+ * Every primitive type has a shortcut to create the trivial type, and + * a builder when custom properties are required. The first line above is + * a shortcut for the second, analogous to the JSON case. + *
Named Types
+ * Avro named types have names, namespace, aliases, and doc. In this API + * these share a common parent, {@link NamespacedBuilder}. + * The builders for named types require a name to be constructed, and optional + * configuration via: + *
  • {@link NamespacedBuilder#doc()}
  • + *
  • {@link NamespacedBuilder#namespace(String)}
  • + *
  • {@link NamespacedBuilder#aliases(String...)}
  • + *
  • {@link PropBuilder#prop(String, Object)}
  • + *

    + * Each named type completes configuration of the optional properties + * with its own method: + *

  • {@link FixedBuilder#size(int)}
  • + *
  • {@link EnumBuilder#symbols(String...)}
  • + *
  • {@link RecordBuilder#fields()}
  • + * Example use of a named type with all optional parameters: + *
    + * .enumeration("Suit").namespace("org.apache.test")
    + *   .aliases("org.apache.test.OldSuit")
    + *   .doc("CardSuits")
    + *   .prop("customProp", "val")
    + *   .symbols("SPADES", "HEARTS", "DIAMONDS", "CLUBS")
    + * 
    + * Which is equivalent to the JSON: + *
    + * { "type":"enum",
    + *   "name":"Suit", "namespace":"org.apache.test",
    + *   "aliases":["org.apache.test.OldSuit"],
    + *   "doc":"Card Suits",
    + *   "customProp":"val",
    + *   "symbols":["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
    + * }
    + * 
    + *
    Nested Types
    + * The Avro nested types, map and array, can have custom properties like + * all avro types, are not named, and must specify a nested type. + * After configuration of optional properties, an array or map + * builds or selects its nested type with {@link ArrayBuilder#items()} + * and {@link MapBuilder#values()}, respectively. + * + *
    Fields
    + * {@link RecordBuilder#fields()} returns a {@link FieldAssembler} for + * defining the fields of the record and completing it. + * Each field must have a name, specified via {@link FieldAssembler#name(String)}, + * which returns a {@link FieldBuilder} for defining aliases, custom properties, + * and documentation of the field. After configuring these optional values for + * a field, the type is selected or built with {@link FieldBuilder#type()}. + *

    + * Fields have default values that must be specified to complete the field. + * {@link FieldDefault#noDefault()} is available for all field types, and + * a specific method is available for each type to use a default, for example + * {@link IntDefault#intDefault(int)} + *

    + * There are field shortcut methods on {@link FieldAssembler} for primitive types. + * These shortcuts create required, optional, and nullable fields, but do not + * support field aliases, doc, or custom properties. + * + *

    Unions
    + * Union types are built via {@link TypeBuilder#unionOf()} or + * {@link FieldTypeBuilder#unionOf()} in the context of type selection. + * This chains together multiple types, in union order. For example: + *
    + * .unionOf()
    + *   .fixed("IPv4").size(4).and()
    + *   .fixed("IPv6").size(16).and()
    + *   .nullType().endUnion()
    + * 
    + * is equivalent to the Avro schema JSON: + *
    + * [
    + *   {"type":"fixed", "name":"IPv4", "size":4},
    + *   {"type":"fixed", "name":"IPv6", "size":16},
    + *   "null"
    + * ]
    + * 
    + * In a field context, the first type of a union defines what default type + * is allowed. + *

    + * Unions have two shortcuts for common cases. nullable() + * creates a union of a type and null. In a field type context, optional() + * is available and creates a union of null and a type, with a null default. + * The below two are equivalent: + *
    + *   .unionOf().intType().and().nullType().endUnion()
    + *   .nullable().intType()
    + * 
    + * The below two field declarations are equivalent: + *
    + *   .name("f").type().unionOf().nullType().and().longType().endUnion().nullDefault()
    + *   .name("f").type().optional().longType()
    + * 
    + * + *
    Explicit Types and Types by Name
    + * Types can also be specified explicitly by passing in a Schema, or by name: + *
    + *   .type(Schema.create(Schema.Type.INT)) // explicitly specified
    + *   .type("MD5")                       // reference by full name or short name
    + *   .type("MD5", "org.apache.avro.test")  // reference by name and namespace
    + * 
    + * When a type is specified by name, and the namespace is absent or null, the + * namespace is inherited from the enclosing context. A namespace will + * propagate as a default to child fields, nested types, or later defined types + * in a union. To specify a name that has no namespace and ignore the inherited + * namespace, set the namespace to "". + *

    + * {@link SchemaBuilder#builder(String)} returns a type builder with a default + * namespace. {@link SchemaBuilder#builder()} returns a type builder with no + * default namespace. + */ +public class SchemaBuilder { + + private SchemaBuilder() { + } + + /** + * Create a builder for Avro schemas. + */ + public static TypeBuilder builder() { + return new TypeBuilder(new SchemaCompletion(), new NameContext()); + } + + /** + * Create a builder for Avro schemas with a default namespace. Types created + * without namespaces will inherit the namespace provided. + */ + public static TypeBuilder builder(String namespace) { + return new TypeBuilder(new SchemaCompletion(), + new NameContext().namespace(namespace)); + } + + /** + * Create a builder for an Avro record with the specified name. + * This is equivalent to: + *

    +     *   builder().record(name);
    +     * 
    + * @param name the record name + */ + public static RecordBuilder record(String name) { + return builder().record(name); + } + + /** + * Create a builder for an Avro enum with the specified name and symbols (values). + * This is equivalent to: + *
    +     *   builder().enumeration(name);
    +     * 
    + * @param name the enum name + */ + public static EnumBuilder enumeration(String name) { + return builder().enumeration(name); + } + + /** + * Create a builder for an Avro fixed type with the specified name and size. + * This is equivalent to: + *
    +     *   builder().fixed(name);
    +     * 
    + * @param name the fixed name + */ + public static FixedBuilder fixed(String name) { + return builder().fixed(name); + } + + /** + * Create a builder for an Avro array + * This is equivalent to: + *
    +     *   builder().array();
    +     * 
    + */ + public static ArrayBuilder array() { + return builder().array(); + } + + /** + * Create a builder for an Avro map + * This is equivalent to: + *
    +     *   builder().map();
    +     * 
    + */ + public static MapBuilder map() { + return builder().map(); + } + + /** + * Create a builder for an Avro union + * This is equivalent to: + *
    +     *   builder().unionOf();
    +     * 
    + */ + public static BaseTypeBuilder> unionOf() { + return builder().unionOf(); + } + + /** + * Create a builder for a union of a type and null. + * This is a shortcut for: + *
    +     *   builder().nullable();
    +     * 
    + * and the following two lines are equivalent: + *
    +     *   nullable().intType();
    +     * 
    + *
    +     *   unionOf().intType().and().nullType().endUnion();
    +     * 
    + */ + public static BaseTypeBuilder nullable() { + return builder().nullable(); + } + + + /** + * An abstract builder for all Avro types. All Avro types + * can have arbitrary string key-value properties. + */ + public static abstract class PropBuilder> { + private Map props = null; + + protected PropBuilder() { + } + + public final S prop(String name, Object val) { + if (!hasProps()) { + props = new HashMap(); + } + props.put(name, val); + return self(); + } + + private boolean hasProps() { + return (props != null); + } + + final T addPropsTo(T jsonable) { + if (hasProps()) { + for (Map.Entry prop : props.entrySet()) { + jsonable.addProp(prop.getKey(), prop.getValue()); + } + } + return jsonable; + } + + /** a self-type for chaining builder subclasses. Concrete subclasses + * must return 'this' **/ + protected abstract S self(); + } + + /** + * An abstract type that provides builder methods for configuring the name, + * doc, and aliases of all Avro types that have names (fields, Fixed, Record, + * and Enum). + *

    + * All Avro named types and fields have 'doc', 'aliases', and 'name' + * components. 'name' is required, and provided to this builder. 'doc' and + * 'alises' are optional. + */ + public static abstract class NamedBuilder> extends + PropBuilder { + private final String name; + private final NameContext names; + private String doc; + private String[] aliases; + + protected NamedBuilder(NameContext names, String name) { + checkRequired(name, "Type must have a name"); + this.names = names; + this.name = name; + } + + /** configure this type's optional documentation string **/ + public final S doc(String doc) { + this.doc = doc; + return self(); + } + + /** configure this type's optional name aliases **/ + public final S aliases(String... aliases) { + this.aliases = aliases; + return self(); + } + + final String doc() { + return doc; + } + + final String name() { + return name; + } + + final NameContext names() { + return names; + } + + final Schema addAliasesTo(Schema schema) { + if (null != aliases) { + for (String alias : aliases) { + schema.addAlias(alias); + } + } + return schema; + } + + final Field addAliasesTo(Field field) { + if (null != aliases) { + for (String alias : aliases) { + field.addAlias(alias); + } + } + return field; + } + } + + /** + * An abstract type that provides builder methods for configuring the + * namespace for all Avro types that have namespaces (Fixed, Record, and + * Enum). + */ + public static abstract class NamespacedBuilder> + extends NamedBuilder { + private final Completion context; + private String namespace; + + protected NamespacedBuilder(Completion context, NameContext names, + String name) { + super(names, name); + this.context = context; + } + + /** + * Set the namespace of this type. To clear the namespace, set empty string. + *

    + * When the namespace is null or unset, the namespace of the type defaults + * to the namespace of the enclosing context. + **/ + public final S namespace(String namespace) { + this.namespace = namespace; + return self(); + } + + final String space() { + if (null == namespace) { + return names().namespace; + } + return namespace; + } + + final Schema completeSchema(Schema schema) { + addPropsTo(schema); + addAliasesTo(schema); + names().put(schema); + return schema; + } + + final Completion context() { + return context; + } + } + + /** + * An abstraction for sharing code amongst all primitive type builders. + */ + private static abstract class PrimitiveBuilder> + extends PropBuilder

    { + private final Completion context; + private final Schema immutable; + + protected PrimitiveBuilder(Completion context, NameContext names, + Schema.Type type) { + this.context = context; + this.immutable = names.getFullname(type.getName()); + } + + private R end() { + Schema schema = immutable; + if (super.hasProps()) { + schema = Schema.create(immutable.getType()); + addPropsTo(schema); + } + return context.complete(schema); + } + } + + /** + * Builds an Avro boolean type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endBoolean()} + **/ + public static final class BooleanBuilder extends + PrimitiveBuilder> { + private BooleanBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.BOOLEAN); + } + + private static BooleanBuilder create(Completion context, + NameContext names) { + return new BooleanBuilder(context, names); + } + + @Override + protected BooleanBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endBoolean() { + return super.end(); + } + } + + /** + * Builds an Avro int type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endInt()} + **/ + public static final class IntBuilder extends + PrimitiveBuilder> { + private IntBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.INT); + } + + private static IntBuilder create(Completion context, + NameContext names) { + return new IntBuilder(context, names); + } + + @Override + protected IntBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endInt() { + return super.end(); + } + } + + /** + * Builds an Avro long type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endLong()} + **/ + public static final class LongBuilder extends + PrimitiveBuilder> { + private LongBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.LONG); + } + + private static LongBuilder create(Completion context, + NameContext names) { + return new LongBuilder(context, names); + } + + @Override + protected LongBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endLong() { + return super.end(); + } + } + + /** + * Builds an Avro float type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endFloat()} + **/ + public static final class FloatBuilder extends + PrimitiveBuilder> { + private FloatBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.FLOAT); + } + + private static FloatBuilder create(Completion context, + NameContext names) { + return new FloatBuilder(context, names); + } + + @Override + protected FloatBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endFloat() { + return super.end(); + } + } + + /** + * Builds an Avro double type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endDouble()} + **/ + public static final class DoubleBuilder extends + PrimitiveBuilder> { + private DoubleBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.DOUBLE); + } + + private static DoubleBuilder create(Completion context, + NameContext names) { + return new DoubleBuilder(context, names); + } + + @Override + protected DoubleBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endDouble() { + return super.end(); + } + } + + /** + * Builds an Avro string type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endString()} + **/ + public static final class StringBldr extends + PrimitiveBuilder> { + private StringBldr(Completion context, NameContext names) { + super(context, names, Schema.Type.STRING); + } + + private static StringBldr create(Completion context, + NameContext names) { + return new StringBldr(context, names); + } + + @Override + protected StringBldr self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endString() { + return super.end(); + } + } + + /** + * Builds an Avro bytes type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endBytes()} + **/ + public static final class BytesBuilder extends + PrimitiveBuilder> { + private BytesBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.BYTES); + } + + private static BytesBuilder create(Completion context, + NameContext names) { + return new BytesBuilder(context, names); + } + + @Override + protected BytesBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endBytes() { + return super.end(); + } + } + + /** + * Builds an Avro null type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endNull()} + **/ + public static final class NullBuilder extends + PrimitiveBuilder> { + private NullBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.NULL); + } + + private static NullBuilder create(Completion context, + NameContext names) { + return new NullBuilder(context, names); + } + + @Override + protected NullBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endNull() { + return super.end(); + } + } + + /** + * Builds an Avro Fixed type with optional properties, namespace, doc, and + * aliases. + *

    + * Set properties with {@link #prop(String, Object)}, namespace with + * {@link #namespace(String)}, doc with {@link #doc(String)}, and aliases with + * {@link #aliases(String[])}. + *

    + * The Fixed schema is finalized when its required size is set via + * {@link #size(int)}. + **/ + public static final class FixedBuilder extends + NamespacedBuilder> { + private FixedBuilder(Completion context, NameContext names, String name) { + super(context, names, name); + } + + private static FixedBuilder create(Completion context, + NameContext names, String name) { + return new FixedBuilder(context, names, name); + } + + @Override + protected FixedBuilder self() { + return this; + } + + /** Configure this fixed type's size, and end its configuration. **/ + public R size(int size) { + Schema schema = Schema.createFixed(name(), super.doc(), space(), size); + completeSchema(schema); + return context().complete(schema); + } + } + + /** + * Builds an Avro Enum type with optional properties, namespace, doc, and + * aliases. + *

    + * Set properties with {@link #prop(String, Object)}, namespace with + * {@link #namespace(String)}, doc with {@link #doc(String)}, and aliases with + * {@link #aliases(String[])}. + *

    + * The Enum schema is finalized when its required symbols are set via + * {@link #symbols(String[])}. + **/ + public static final class EnumBuilder extends + NamespacedBuilder> { + private EnumBuilder(Completion context, NameContext names, String name) { + super(context, names, name); + } + + private static EnumBuilder create(Completion context, + NameContext names, String name) { + return new EnumBuilder(context, names, name); + } + + @Override + protected EnumBuilder self() { + return this; + } + + /** Configure this enum type's symbols, and end its configuration. **/ + public R symbols(String... symbols) { + Schema schema = Schema.createEnum(name(), doc(), space(), + Arrays.asList(symbols)); + completeSchema(schema); + return context().complete(schema); + } + + } + + /** + * Builds an Avro Map type with optional properties. + *

    + * Set properties with {@link #prop(String, Object)}. + *

    + * The Map schema's properties are finalized when {@link #values()} or + * {@link #values(Schema)} is called. + **/ + public static final class MapBuilder extends PropBuilder> { + private final Completion context; + private final NameContext names; + + private MapBuilder(Completion context, NameContext names) { + this.context = context; + this.names = names; + } + + private static MapBuilder create(Completion context, + NameContext names) { + return new MapBuilder(context, names); + } + + @Override + protected MapBuilder self() { + return this; + } + + /** + * Return a type builder for configuring the map's nested values schema. + * This builder will return control to the map's enclosing context when + * complete. + **/ + public TypeBuilder values() { + return new TypeBuilder(new MapCompletion(this, context), names); + } + + /** + * Complete configuration of this map, setting the schema of the map values + * to the schema provided. Returns control to the enclosing context. + **/ + public R values(Schema valueSchema) { + return new MapCompletion(this, context).complete(valueSchema); + } + } + + /** + * Builds an Avro Array type with optional properties. + *

    + * Set properties with {@link #prop(String, Object)}. + *

    + * The Array schema's properties are finalized when {@link #items()} or + * {@link #items(Schema)} is called. + **/ + public static final class ArrayBuilder extends + PropBuilder> { + private final Completion context; + private final NameContext names; + + public ArrayBuilder(Completion context, NameContext names) { + this.context = context; + this.names = names; + } + + private static ArrayBuilder create(Completion context, + NameContext names) { + return new ArrayBuilder(context, names); + } + + @Override + protected ArrayBuilder self() { + return this; + } + + /** + * Return a type builder for configuring the array's nested items schema. + * This builder will return control to the array's enclosing context when + * complete. + **/ + public TypeBuilder items() { + return new TypeBuilder(new ArrayCompletion(this, context), names); + } + + /** + * Complete configuration of this array, setting the schema of the array + * items to the schema provided. Returns control to the enclosing context. + **/ + public R items(Schema itemsSchema) { + return new ArrayCompletion(this, context).complete(itemsSchema); + } + } + + /** + * internal class for passing the naming context around. This allows for the + * following: + *

  • Cache and re-use primitive schemas when they do not set + * properties.
  • + *
  • Provide a default namespace for nested contexts (as + * the JSON Schema spec does).
  • + *
  • Allow previously defined named types or primitive types + * to be referenced by name.
  • + **/ + private static class NameContext { + private static final Set PRIMITIVES = new HashSet(); + + { + PRIMITIVES.add("null"); + PRIMITIVES.add("boolean"); + PRIMITIVES.add("int"); + PRIMITIVES.add("long"); + PRIMITIVES.add("float"); + PRIMITIVES.add("double"); + PRIMITIVES.add("bytes"); + PRIMITIVES.add("string"); + } + + private final HashMap schemas; + private final String namespace; + + private NameContext() { + this.schemas = new HashMap(); + this.namespace = null; + schemas.put("null", Schema.create(Schema.Type.NULL)); + schemas.put("boolean", Schema.create(Schema.Type.BOOLEAN)); + schemas.put("int", Schema.create(Schema.Type.INT)); + schemas.put("long", Schema.create(Schema.Type.LONG)); + schemas.put("float", Schema.create(Schema.Type.FLOAT)); + schemas.put("double", Schema.create(Schema.Type.DOUBLE)); + schemas.put("bytes", Schema.create(Schema.Type.BYTES)); + schemas.put("string", Schema.create(Schema.Type.STRING)); + } + + private NameContext(HashMap schemas, String namespace) { + this.schemas = schemas; + this.namespace = "".equals(namespace) ? null : namespace; + } + + private NameContext namespace(String namespace) { + return new NameContext(schemas, namespace); + } + + private Schema get(String name, String namespace) { + return getFullname(resolveName(name, namespace)); + } + + private Schema getFullname(String fullName) { + Schema schema = schemas.get(fullName); + if (schema == null) { + throw new SchemaParseException("Undefined name: " + fullName); + } + return schema; + } + + private void put(Schema schema) { + String fullName = schema.getFullName(); + if (schemas.containsKey(fullName)) { + throw new SchemaParseException("Can't redefine: " + fullName); + } + schemas.put(fullName, schema); + } + + private String resolveName(String name, String space) { + if (PRIMITIVES.contains(name) && space == null) { + return name; + } + int lastDot = name.lastIndexOf('.'); + if (lastDot < 0) { // short name + if (space == null) { + space = namespace; + } + if (space != null && !"".equals(space)) { + return space + "." + name; + } + } + return name; + } + } + + /** + * A common API for building types within a context. BaseTypeBuilder can build + * all types other than Unions. {@link TypeBuilder} can additionally build + * Unions. + *

    + * The builder has two contexts: + *

  • A naming context provides a default namespace and allows for previously + * defined named types to be referenced from {@link #type(String)}
  • + *
  • A completion context representing the scope that the builder was + * created in. A builder created in a nested context (for example, + * {@link MapBuilder#values()} will have a completion context assigned by the + * {@link MapBuilder}
  • + **/ + public static class BaseTypeBuilder { + private final Completion context; + private final NameContext names; + + private BaseTypeBuilder(Completion context, NameContext names) { + this.context = context; + this.names = names; + } + + /** Use the schema provided as the type. **/ + public final R type(Schema schema) { + return context.complete(schema); + } + + /** + * Look up the type by name. This type must be previously defined in the + * context of this builder. + *

    + * The name may be fully qualified or a short name. If it is a short name, + * the default namespace of the current context will additionally be + * searched. + **/ + public final R type(String name) { + return type(name, null); + } + + /** + * Look up the type by name and namespace. This type must be previously + * defined in the context of this builder. + *

    + * The name may be fully qualified or a short name. If it is a fully + * qualified name, the namespace provided is ignored. If it is a short name, + * the namespace provided is used if not null, else the default namespace of + * the current context will be used. + **/ + public final R type(String name, String namespace) { + return type(names.get(name, namespace)); + } + + /** + * A plain boolean type without custom properties. This is equivalent to: + *

    +         * booleanBuilder().endBoolean();
    +         * 
    + */ + public final R booleanType() { + return booleanBuilder().endBoolean(); + } + + /** + * Build a boolean type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #booleanType()}. + */ + public final BooleanBuilder booleanBuilder() { + return BooleanBuilder.create(context, names); + } + + /** + * A plain int type without custom properties. This is equivalent to: + *
    +         * intBuilder().endInt();
    +         * 
    + */ + public final R intType() { + return intBuilder().endInt(); + } + + /** + * Build an int type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #intType()}. + */ + public final IntBuilder intBuilder() { + return IntBuilder.create(context, names); + } + + /** + * A plain long type without custom properties. This is equivalent to: + *
    +         * longBuilder().endLong();
    +         * 
    + */ + public final R longType() { + return longBuilder().endLong(); + } + + /** + * Build a long type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #longType()}. + */ + public final LongBuilder longBuilder() { + return LongBuilder.create(context, names); + } + + /** + * A plain float type without custom properties. This is equivalent to: + *
    +         * floatBuilder().endFloat();
    +         * 
    + */ + public final R floatType() { + return floatBuilder().endFloat(); + } + + /** + * Build a float type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #floatType()}. + */ + public final FloatBuilder floatBuilder() { + return FloatBuilder.create(context, names); + } + + /** + * A plain double type without custom properties. This is equivalent to: + *
    +         * doubleBuilder().endDouble();
    +         * 
    + */ + public final R doubleType() { + return doubleBuilder().endDouble(); + } + + /** + * Build a double type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #doubleType()}. + */ + public final DoubleBuilder doubleBuilder() { + return DoubleBuilder.create(context, names); + } + + /** + * A plain string type without custom properties. This is equivalent to: + *
    +         * stringBuilder().endString();
    +         * 
    + */ + public final R stringType() { + return stringBuilder().endString(); + } + + /** + * Build a string type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #stringType()}. + */ + public final StringBldr stringBuilder() { + return StringBldr.create(context, names); + } + + /** + * A plain bytes type without custom properties. This is equivalent to: + *
    +         * bytesBuilder().endBytes();
    +         * 
    + */ + public final R bytesType() { + return bytesBuilder().endBytes(); + } + + /** + * Build a bytes type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #bytesType()}. + */ + public final BytesBuilder bytesBuilder() { + return BytesBuilder.create(context, names); + } + + /** + * A plain null type without custom properties. This is equivalent to: + *
    +         * nullBuilder().endNull();
    +         * 
    + */ + public final R nullType() { + return nullBuilder().endNull(); + } + + /** + * Build a null type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #nullType()}. + */ + public final NullBuilder nullBuilder() { + return NullBuilder.create(context, names); + } + + /** Build an Avro map type Example usage: + *
    +         * map().values().intType()
    +         * 
    + * Equivalent to Avro JSON Schema: + *
    +         * {"type":"map", "values":"int"}
    +         * 
    + **/ + public final MapBuilder map() { + return MapBuilder.create(context, names); + } + + /** Build an Avro array type Example usage: + *
    +         * array().items().longType()
    +         * 
    + * Equivalent to Avro JSON Schema: + *
    +         * {"type":"array", "values":"long"}
    +         * 
    + **/ + public final ArrayBuilder array() { + return ArrayBuilder.create(context, names); + } + + /** Build an Avro fixed type. Example usage: + *
    +         * fixed("com.foo.IPv4").size(4)
    +         * 
    + * Equivalent to Avro JSON Schema: + *
    +         * {"type":"fixed", "name":"com.foo.IPv4", "size":4}
    +         * 
    + **/ + public final FixedBuilder fixed(String name) { + return FixedBuilder.create(context, names, name); + } + + /** Build an Avro enum type. Example usage: + *
    +         * enumeration("Suits").namespace("org.cards").doc("card suit names")
    +         *   .symbols("HEART", "SPADE", "DIAMOND", "CLUB")
    +         * 
    + * Equivalent to Avro JSON Schema: + *
    +         * {"type":"enum", "name":"Suits", "namespace":"org.cards",
    +         *  "doc":"card suit names", "symbols":[
    +         *    "HEART", "SPADE", "DIAMOND", "CLUB"]}
    +         * 
    + **/ + public final EnumBuilder enumeration(String name) { + return EnumBuilder.create(context, names, name); + } + + /** Build an Avro record type. Example usage: + *
    +         * record("com.foo.Foo").fields()
    +         *   .name("field1").typeInt().intDefault(1)
    +         *   .name("field2").typeString().noDefault()
    +         *   .name("field3").optional().typeFixed("FooFixed").size(4)
    +         *   .endRecord()
    +         * 
    + * Equivalent to Avro JSON Schema: + *
    +         * {"type":"record", "name":"com.foo.Foo", "fields": [
    +         *   {"name":"field1", "type":"int", "default":1},
    +         *   {"name":"field2", "type":"string"},
    +         *   {"name":"field3", "type":[
    +         *     null, {"type":"fixed", "name":"FooFixed", "size":4}
    +         *     ]}
    +         *   ]}
    +         * 
    + **/ + public final RecordBuilder record(String name) { + return RecordBuilder.create(context, names, name); + } + + /** Build an Avro union schema type. Example usage: + *
    unionOf().stringType().and().bytesType().endUnion()
    + **/ + protected BaseTypeBuilder> unionOf() { + return UnionBuilder.create(context, names); + } + + /** A shortcut for building a union of a type and null. + *

    + * For example, the code snippets below are equivalent: + *

    nullable().booleanType()
    + *
    unionOf().booleanType().and().nullType().endUnion()
    + **/ + protected BaseTypeBuilder nullable() { + return new BaseTypeBuilder(new NullableCompletion(context), names); + } + + } + + /** A Builder for creating any Avro schema type. + **/ + public static final class TypeBuilder extends BaseTypeBuilder { + private TypeBuilder(Completion context, NameContext names) { + super(context, names); + } + + @Override + public BaseTypeBuilder> unionOf() { + return super.unionOf(); + } + + @Override + public BaseTypeBuilder nullable() { + return super.nullable(); + } + } + + /** A special builder for unions. Unions cannot nest unions directly **/ + private static final class UnionBuilder extends + BaseTypeBuilder> { + private UnionBuilder(Completion context, NameContext names) { + this(context, names, new ArrayList()); + } + + private static UnionBuilder create(Completion context, NameContext names) { + return new UnionBuilder(context, names); + } + + private UnionBuilder(Completion context, NameContext names, List schemas) { + super(new UnionCompletion(context, names, schemas), names); + } + } + + /** + * A special Builder for Record fields. The API is very similar to + * {@link BaseTypeBuilder}. However, fields have their own names, properties, + * and default values. + *

    + * The methods on this class create builder instances that return their + * control to the {@link FieldAssembler} of the enclosing record context after + * configuring a default for the field. + *

    + * For example, an int field with default value 1: + *

    +     * intSimple().withDefault(1);
    +     * 
    + * or an array with items that are optional int types: + *
    +     * array().items().optional().intType();
    +     * 
    + */ + public static class BaseFieldTypeBuilder { + protected final FieldBuilder bldr; + protected final NameContext names; + private final CompletionWrapper wrapper; + + protected BaseFieldTypeBuilder(FieldBuilder bldr, CompletionWrapper wrapper) { + this.bldr = bldr; + this.names = bldr.names(); + this.wrapper = wrapper; + } + + /** + * A plain boolean type without custom properties. This is equivalent to: + *
    +         * booleanBuilder().endBoolean();
    +         * 
    + */ + public final BooleanDefault booleanType() { + return booleanBuilder().endBoolean(); + } + + /** + * Build a boolean type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #booleanType()}. + */ + public final BooleanBuilder> booleanBuilder() { + return BooleanBuilder.create(wrap(new BooleanDefault(bldr)), names); + } + + /** + * A plain int type without custom properties. This is equivalent to: + *
    +         * intBuilder().endInt();
    +         * 
    + */ + public final IntDefault intType() { + return intBuilder().endInt(); + } + + /** + * Build an int type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #intType()}. + */ + public final IntBuilder> intBuilder() { + return IntBuilder.create(wrap(new IntDefault(bldr)), names); + } + + /** + * A plain long type without custom properties. This is equivalent to: + *
    +         * longBuilder().endLong();
    +         * 
    + */ + public final LongDefault longType() { + return longBuilder().endLong(); + } + + /** + * Build a long type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #longType()}. + */ + public final LongBuilder> longBuilder() { + return LongBuilder.create(wrap(new LongDefault(bldr)), names); + } + + /** + * A plain float type without custom properties. This is equivalent to: + *
    +         * floatBuilder().endFloat();
    +         * 
    + */ + public final FloatDefault floatType() { + return floatBuilder().endFloat(); + } + + /** + * Build a float type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #floatType()}. + */ + public final FloatBuilder> floatBuilder() { + return FloatBuilder.create(wrap(new FloatDefault(bldr)), names); + } + + /** + * A plain double type without custom properties. This is equivalent to: + *
    +         * doubleBuilder().endDouble();
    +         * 
    + */ + public final DoubleDefault doubleType() { + return doubleBuilder().endDouble(); + } + + /** + * Build a double type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #doubleType()}. + */ + public final DoubleBuilder> doubleBuilder() { + return DoubleBuilder.create(wrap(new DoubleDefault(bldr)), names); + } + + /** + * A plain string type without custom properties. This is equivalent to: + *
    +         * stringBuilder().endString();
    +         * 
    + */ + public final StringDefault stringType() { + return stringBuilder().endString(); + } + + /** + * Build a string type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #stringType()}. + */ + public final StringBldr> stringBuilder() { + return StringBldr.create(wrap(new StringDefault(bldr)), names); + } + + /** + * A plain bytes type without custom properties. This is equivalent to: + *
    +         * bytesBuilder().endBytes();
    +         * 
    + */ + public final BytesDefault bytesType() { + return bytesBuilder().endBytes(); + } + + /** + * Build a bytes type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #bytesType()}. + */ + public final BytesBuilder> bytesBuilder() { + return BytesBuilder.create(wrap(new BytesDefault(bldr)), names); + } + + /** + * A plain null type without custom properties. This is equivalent to: + *
    +         * nullBuilder().endNull();
    +         * 
    + */ + public final NullDefault nullType() { + return nullBuilder().endNull(); + } + + /** + * Build a null type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #nullType()}. + */ + public final NullBuilder> nullBuilder() { + return NullBuilder.create(wrap(new NullDefault(bldr)), names); + } + + /** Build an Avro map type **/ + public final MapBuilder> map() { + return MapBuilder.create(wrap(new MapDefault(bldr)), names); + } + + /** Build an Avro array type **/ + public final ArrayBuilder> array() { + return ArrayBuilder.create(wrap(new ArrayDefault(bldr)), names); + } + + /** Build an Avro fixed type. **/ + public final FixedBuilder> fixed(String name) { + return FixedBuilder.create(wrap(new FixedDefault(bldr)), names, name); + } + + /** Build an Avro enum type. **/ + public final EnumBuilder> enumeration(String name) { + return EnumBuilder.create(wrap(new EnumDefault(bldr)), names, name); + } + + /** Build an Avro record type. **/ + public final RecordBuilder> record(String name) { + return RecordBuilder.create(wrap(new RecordDefault(bldr)), names, name); + } + + private Completion wrap( + Completion completion) { + if (wrapper != null) { + return wrapper.wrap(completion); + } + return completion; + } + } + + /** FieldTypeBuilder adds {@link #unionOf()}, {@link #nullable()}, and {@link #optional()} + * to BaseFieldTypeBuilder. **/ + public static final class FieldTypeBuilder extends BaseFieldTypeBuilder { + private FieldTypeBuilder(FieldBuilder bldr) { + super(bldr, null); + } + + /** Build an Avro union schema type. **/ + public UnionFieldTypeBuilder unionOf() { + return new UnionFieldTypeBuilder(bldr); + } + + /** + * A shortcut for building a union of a type and null, with an optional default + * value of the non-null type. + *

    + * For example, the two code snippets below are equivalent: + *

    nullable().booleanType().booleanDefault(true)
    + *
    unionOf().booleanType().and().nullType().endUnion().booleanDefault(true)
    + **/ + public BaseFieldTypeBuilder nullable() { + return new BaseFieldTypeBuilder(bldr, new NullableCompletionWrapper()); + } + + /** + * A shortcut for building a union of null and a type, with a null default. + *

    + * For example, the two code snippets below are equivalent: + *

    optional().booleanType()
    + *
    unionOf().nullType().and().booleanType().endUnion().nullDefault()
    + */ + public BaseTypeBuilder> optional() { + return new BaseTypeBuilder>( + new OptionalCompletion(bldr), names); + } + } + + /** Builder for a union field. The first type in the union corresponds + * to the possible default value type. + */ + public static final class UnionFieldTypeBuilder { + private final FieldBuilder bldr; + private final NameContext names; + + private UnionFieldTypeBuilder(FieldBuilder bldr) { + this.bldr = bldr; + this.names = bldr.names(); + } + + /** + * A plain boolean type without custom properties. This is equivalent to: + *
    +         * booleanBuilder().endBoolean();
    +         * 
    + */ + public UnionAccumulator> booleanType() { + return booleanBuilder().endBoolean(); + } + + /** + * Build a boolean type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #booleanType()}. + */ + public BooleanBuilder>> booleanBuilder() { + return BooleanBuilder.create(completion(new BooleanDefault(bldr)), names); + } + + /** + * A plain int type without custom properties. This is equivalent to: + *
    +         * intBuilder().endInt();
    +         * 
    + */ + public UnionAccumulator> intType() { + return intBuilder().endInt(); + } + + /** + * Build an int type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #intType()}. + */ + public IntBuilder>> intBuilder() { + return IntBuilder.create(completion(new IntDefault(bldr)), names); + } + + /** + * A plain long type without custom properties. This is equivalent to: + *
    +         * longBuilder().endLong();
    +         * 
    + */ + public UnionAccumulator> longType() { + return longBuilder().endLong(); + } + + /** + * Build a long type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #longType()}. + */ + public LongBuilder>> longBuilder() { + return LongBuilder.create(completion(new LongDefault(bldr)), names); + } + + /** + * A plain float type without custom properties. This is equivalent to: + *
    +         * floatBuilder().endFloat();
    +         * 
    + */ + public UnionAccumulator> floatType() { + return floatBuilder().endFloat(); + } + + /** + * Build a float type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #floatType()}. + */ + public FloatBuilder>> floatBuilder() { + return FloatBuilder.create(completion(new FloatDefault(bldr)), names); + } + + /** + * A plain double type without custom properties. This is equivalent to: + *
    +         * doubleBuilder().endDouble();
    +         * 
    + */ + public UnionAccumulator> doubleType() { + return doubleBuilder().endDouble(); + } + + /** + * Build a double type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #doubleType()}. + */ + public DoubleBuilder>> doubleBuilder() { + return DoubleBuilder.create(completion(new DoubleDefault(bldr)), names); + } + + /** + * A plain string type without custom properties. This is equivalent to: + *
    +         * stringBuilder().endString();
    +         * 
    + */ + public UnionAccumulator> stringType() { + return stringBuilder().endString(); + } + + /** + * Build a string type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #stringType()}. + */ + public StringBldr>> stringBuilder() { + return StringBldr.create(completion(new StringDefault(bldr)), names); + } + + /** + * A plain bytes type without custom properties. This is equivalent to: + *
    +         * bytesBuilder().endBytes();
    +         * 
    + */ + public UnionAccumulator> bytesType() { + return bytesBuilder().endBytes(); + } + + /** + * Build a bytes type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #bytesType()}. + */ + public BytesBuilder>> bytesBuilder() { + return BytesBuilder.create(completion(new BytesDefault(bldr)), names); + } + + /** + * A plain null type without custom properties. This is equivalent to: + *
    +         * nullBuilder().endNull();
    +         * 
    + */ + public UnionAccumulator> nullType() { + return nullBuilder().endNull(); + } + + /** + * Build a null type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #nullType()}. + */ + public NullBuilder>> nullBuilder() { + return NullBuilder.create(completion(new NullDefault(bldr)), names); + } + + /** Build an Avro map type **/ + public MapBuilder>> map() { + return MapBuilder.create(completion(new MapDefault(bldr)), names); + } + + /** Build an Avro array type **/ + public ArrayBuilder>> array() { + return ArrayBuilder.create(completion(new ArrayDefault(bldr)), names); + } + + /** Build an Avro fixed type. **/ + public FixedBuilder>> fixed(String name) { + return FixedBuilder.create(completion(new FixedDefault(bldr)), names, name); + } + + /** Build an Avro enum type. **/ + public EnumBuilder>> enumeration(String name) { + return EnumBuilder.create(completion(new EnumDefault(bldr)), names, name); + } + + /** Build an Avro record type. **/ + public RecordBuilder>> record(String name) { + return RecordBuilder.create(completion(new RecordDefault(bldr)), names, name); + } + + private UnionCompletion completion(Completion context) { + return new UnionCompletion(context, names, new ArrayList()); + } + } + + public final static class RecordBuilder extends + NamespacedBuilder> { + private RecordBuilder(Completion context, NameContext names, String name) { + super(context, names, name); + } + + private static RecordBuilder create(Completion context, + NameContext names, String name) { + return new RecordBuilder(context, names, name); + } + + @Override + protected RecordBuilder self() { + return this; + } + + public FieldAssembler fields() { + Schema record = Schema.createRecord(name(), doc(), space(), false); + // place the record in the name context, fields yet to be set. + completeSchema(record); + return new FieldAssembler( + context(), names().namespace(record.getNamespace()), record); + } + } + + public final static class FieldAssembler { + private final List fields = new ArrayList(); + private final Completion context; + private final NameContext names; + private final Schema record; + + private FieldAssembler(Completion context, NameContext names, Schema record) { + this.context = context; + this.names = names; + this.record = record; + } + + /** + * Add a field with the given name. + * @return A {@link FieldBuilder} for the given name. + */ + public FieldBuilder name(String fieldName) { + return new FieldBuilder(this, names, fieldName); + } + + /** + * Shortcut for creating a boolean field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().booleanType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredBoolean(String fieldName) { + return name(fieldName).type().booleanType().noDefault(); + } + + /** + * Shortcut for creating an optional boolean field: a union of null and + * boolean with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().booleanType()
    +         * 
    + */ + public FieldAssembler optionalBoolean(String fieldName) { + return name(fieldName).type().optional().booleanType(); + } + + /** + * Shortcut for creating a nullable boolean field: a union of boolean and + * null with an boolean default. + *

    + * This is equivalent to: + * + *

    +         * name(fieldName).type().nullable().booleanType().booleanDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableBoolean(String fieldName, boolean defaultVal) { + return name(fieldName) + .type().nullable().booleanType().booleanDefault(defaultVal); + } + + /** + * Shortcut for creating an int field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().intType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredInt(String fieldName) { + return name(fieldName).type().intType().noDefault(); + } + + /** + * Shortcut for creating an optional int field: a union of null and int + * with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().intType()
    +         * 
    + */ + public FieldAssembler optionalInt(String fieldName) { + return name(fieldName).type().optional().intType(); + } + + /** + * Shortcut for creating a nullable int field: a union of int and null + * with an int default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().nullable().intType().intDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableInt(String fieldName, int defaultVal) { + return name(fieldName).type().nullable().intType().intDefault(defaultVal); + } + + /** + * Shortcut for creating a long field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().longType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredLong(String fieldName) { + return name(fieldName).type().longType().noDefault(); + } + + /** + * Shortcut for creating an optional long field: a union of null and long + * with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().longType()
    +         * 
    + */ + public FieldAssembler optionalLong(String fieldName) { + return name(fieldName).type().optional().longType(); + } + + /** + * Shortcut for creating a nullable long field: a union of long and null + * with a long default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().nullable().longType().longDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableLong(String fieldName, long defaultVal) { + return name(fieldName).type().nullable().longType().longDefault(defaultVal); + } + + /** + * Shortcut for creating a float field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().floatType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredFloat(String fieldName) { + return name(fieldName).type().floatType().noDefault(); + } + + /** + * Shortcut for creating an optional float field: a union of null and float + * with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().floatType()
    +         * 
    + */ + public FieldAssembler optionalFloat(String fieldName) { + return name(fieldName).type().optional().floatType(); + } + + /** + * Shortcut for creating a nullable float field: a union of float and null + * with a float default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().nullable().floatType().floatDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableFloat(String fieldName, float defaultVal) { + return name(fieldName).type().nullable().floatType().floatDefault(defaultVal); + } + + /** + * Shortcut for creating a double field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().doubleType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredDouble(String fieldName) { + return name(fieldName).type().doubleType().noDefault(); + } + + /** + * Shortcut for creating an optional double field: a union of null and double + * with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().doubleType()
    +         * 
    + */ + public FieldAssembler optionalDouble(String fieldName) { + return name(fieldName).type().optional().doubleType(); + } + + /** + * Shortcut for creating a nullable double field: a union of double and null + * with a double default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().nullable().doubleType().doubleDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableDouble(String fieldName, double defaultVal) { + return name(fieldName).type().nullable().doubleType().doubleDefault(defaultVal); + } + + /** + * Shortcut for creating a string field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().stringType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredString(String fieldName) { + return name(fieldName).type().stringType().noDefault(); + } + + /** + * Shortcut for creating an optional string field: a union of null and string + * with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().stringType()
    +         * 
    + */ + public FieldAssembler optionalString(String fieldName) { + return name(fieldName).type().optional().stringType(); + } + + /** + * Shortcut for creating a nullable string field: a union of string and null + * with a string default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().nullable().stringType().stringDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableString(String fieldName, String defaultVal) { + return name(fieldName).type().nullable().stringType().stringDefault(defaultVal); + } + + /** + * Shortcut for creating a bytes field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().bytesType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredBytes(String fieldName) { + return name(fieldName).type().bytesType().noDefault(); + } + + /** + * Shortcut for creating an optional bytes field: a union of null and bytes + * with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().bytesType()
    +         * 
    + */ + public FieldAssembler optionalBytes(String fieldName) { + return name(fieldName).type().optional().bytesType(); + } + + /** + * Shortcut for creating a nullable bytes field: a union of bytes and null + * with a bytes default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().nullable().bytesType().bytesDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableBytes(String fieldName, byte[] defaultVal) { + return name(fieldName).type().nullable().bytesType().bytesDefault(defaultVal); + } + + /** + * End adding fields to this record, returning control + * to the context that this record builder was created in. + */ + public R endRecord() { + record.setFields(fields); + return context.complete(record); + } + + private FieldAssembler addField(Field field) { + fields.add(field); + return this; + } + + } + + /** + * Builds a Field in the context of a {@link FieldAssembler}. + * + * Usage is to first configure any of the optional parameters and then to call one + * of the type methods to complete the field. For example + *
    +     *   .namespace("org.apache.example").orderDecending().type()
    +     * 
    + * Optional parameters for a field are namespace, doc, order, and aliases. + */ + public final static class FieldBuilder extends + NamedBuilder> { + private final FieldAssembler fields; + private Schema.Field.Order order = Schema.Field.Order.ASCENDING; + + private FieldBuilder(FieldAssembler fields, NameContext names, String name) { + super(names, name); + this.fields = fields; + } + + /** Set this field to have ascending order. Ascending is the default **/ + public FieldBuilder orderAscending() { + order = Schema.Field.Order.ASCENDING; + return self(); + } + + /** Set this field to have decending order. Decending is the default **/ + public FieldBuilder orderDescending() { + order = Schema.Field.Order.DESCENDING; + return self(); + } + + /** Set this field to ignore order. **/ + public FieldBuilder orderIgnore() { + order = Schema.Field.Order.IGNORE; + return self(); + } + + /** + * Final step in configuring this field, finalizing name, namespace, alias, + * and order. + * @return A builder for the field's type and default value. + */ + public FieldTypeBuilder type() { + return new FieldTypeBuilder(this); + } + + /** + * Final step in configuring this field, finalizing name, namespace, alias, + * and order. Sets the field's type to the provided schema, returns a + * {@link GenericDefault}. + */ + public GenericDefault type(Schema type) { + return new GenericDefault(this, type); + } + + /** + * Final step in configuring this field, finalizing name, namespace, alias, + * and order. Sets the field's type to the schema by name reference. + *

    + * The name must correspond with a named schema that has already been + * created in the context of this builder. The name may be a fully qualified + * name, or a short name. If it is a short name, the namespace context of + * this builder will be used. + *

    + * The name and namespace context rules are the same as the Avro schema JSON + * specification. + */ + public GenericDefault type(String name) { + return type(name, null); + } + + /** + * Final step in configuring this field, finalizing name, namespace, alias, + * and order. Sets the field's type to the schema by name reference. + *

    + * The name must correspond with a named schema that has already been + * created in the context of this builder. The name may be a fully qualified + * name, or a short name. If it is a full name, the namespace is ignored. If + * it is a short name, the namespace provided is used. If the namespace + * provided is null, the namespace context of this builder will be used. + *

    + * The name and namespace context rules are the same as the Avro schema JSON + * specification. + */ + public GenericDefault type(String name, String namespace) { + Schema schema = names().get(name, namespace); + return type(schema); + } + + private FieldAssembler completeField(Schema schema) { + return completeField(schema, null); + } + + private FieldAssembler completeField(Schema schema, Object defaultVal) { + Field field = new Field(name(), schema, doc(), defaultVal, order); + addPropsTo(field); + addAliasesTo(field); + return fields.addField(field); + } + + @Override + protected FieldBuilder self() { + return this; + } + } + + /** Abstract base class for field defaults. **/ + public static abstract class FieldDefault> extends Completion { + private final FieldBuilder field; + private Schema schema; + + FieldDefault(FieldBuilder field) { + this.field = field; + } + + /** Completes this field with no default value **/ + public final FieldAssembler noDefault() { + return field.completeField(schema); + } + + private FieldAssembler usingDefault(Object defaultVal) { + return field.completeField(schema, defaultVal); + } + + @Override + final S complete(Schema schema) { + this.schema = schema; + return self(); + } + + abstract S self(); + } + + /** Choose whether to use a default value for the field or not. **/ + public static class BooleanDefault extends FieldDefault> { + private BooleanDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided **/ + public final FieldAssembler booleanDefault(boolean defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final BooleanDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class IntDefault extends FieldDefault> { + private IntDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided **/ + public final FieldAssembler intDefault(int defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final IntDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class LongDefault extends FieldDefault> { + private LongDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided **/ + public final FieldAssembler longDefault(long defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final LongDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class FloatDefault extends FieldDefault> { + private FloatDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided **/ + public final FieldAssembler floatDefault(float defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final FloatDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class DoubleDefault extends FieldDefault> { + private DoubleDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided **/ + public final FieldAssembler doubleDefault(double defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final DoubleDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class StringDefault extends FieldDefault> { + private StringDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided. Cannot be null. **/ + public final FieldAssembler stringDefault(String defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final StringDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class BytesDefault extends FieldDefault> { + private BytesDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler bytesDefault(byte[] defaultVal) { + return super.usingDefault(ByteBuffer.wrap(defaultVal)); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler bytesDefault(ByteBuffer defaultVal) { + return super.usingDefault(defaultVal); + } + + /** Completes this field with the default value provided, cannot be null. + * The string is interpreted as a byte[], with each character code point + * value equalling the byte value, as in the Avro spec JSON default. **/ + public final FieldAssembler bytesDefault(String defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final BytesDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class NullDefault extends FieldDefault> { + private NullDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with a default value of null **/ + public final FieldAssembler nullDefault() { + return super.usingDefault(null); + } + + @Override + final NullDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class MapDefault extends FieldDefault> { + private MapDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler mapDefault(Map defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final MapDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class ArrayDefault extends FieldDefault> { + private ArrayDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler arrayDefault(List defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final ArrayDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class FixedDefault extends FieldDefault> { + private FixedDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler fixedDefault(byte[] defaultVal) { + return super.usingDefault(ByteBuffer.wrap(defaultVal)); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler fixedDefault(ByteBuffer defaultVal) { + return super.usingDefault(defaultVal); + } + + /** Completes this field with the default value provided, cannot be null. + * The string is interpreted as a byte[], with each character code point + * value equalling the byte value, as in the Avro spec JSON default. **/ + public final FieldAssembler fixedDefault(String defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final FixedDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class EnumDefault extends FieldDefault> { + private EnumDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler enumDefault(String defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final EnumDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class RecordDefault extends FieldDefault> { + private RecordDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler recordDefault(GenericRecord defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final RecordDefault self() { + return this; + } + } + + public final static class GenericDefault { + private final FieldBuilder field; + private final Schema schema; + + private GenericDefault(FieldBuilder field, Schema schema) { + this.field = field; + this.schema = schema; + } + + /** Do not use a default value for this field. **/ + public FieldAssembler noDefault() { + return field.completeField(schema); + } + + /** Completes this field with the default value provided. + * The value must conform to the schema of the field. **/ + public FieldAssembler withDefault(Object defaultVal) { + return field.completeField(schema, defaultVal); + } + } + + /** + * Completion is for internal builder use, all subclasses are private. + * + * Completion is an object that takes a Schema and returns some result. + */ + private abstract static class Completion { + abstract R complete(Schema schema); + } + + private static class SchemaCompletion extends Completion { + @Override + protected Schema complete(Schema schema) { + return schema; + } + } + + private static final Schema NULL_SCHEMA = Schema.create(Schema.Type.NULL); + + private static class NullableCompletion extends Completion { + private final Completion context; + + private NullableCompletion(Completion context) { + this.context = context; + } + + @Override + protected R complete(Schema schema) { + // wrap the schema as a union of the schema and null + Schema nullable = Schema.createUnion(Arrays.asList(schema, NULL_SCHEMA)); + return context.complete(nullable); + } + } + + private static class OptionalCompletion extends Completion> { + private final FieldBuilder bldr; + + public OptionalCompletion(FieldBuilder bldr) { + this.bldr = bldr; + } + + @Override + protected FieldAssembler complete(Schema schema) { + // wrap the schema as a union of null and the schema + Schema optional = Schema.createUnion(Arrays.asList(NULL_SCHEMA, schema)); + return bldr.completeField(optional, (Object) null); + } + } + + private abstract static class CompletionWrapper { + abstract Completion wrap(Completion completion); + } + + private static final class NullableCompletionWrapper extends CompletionWrapper { + @Override + Completion wrap(Completion completion) { + return new NullableCompletion(completion); + } + } + + private static abstract class NestedCompletion extends Completion { + private final Completion context; + private final PropBuilder assembler; + + private NestedCompletion(PropBuilder assembler, Completion context) { + this.context = context; + this.assembler = assembler; + } + + @Override + protected final R complete(Schema schema) { + Schema outer = outerSchema(schema); + assembler.addPropsTo(outer); + return context.complete(outer); + } + + protected abstract Schema outerSchema(Schema inner); + } + + private static class MapCompletion extends NestedCompletion { + private MapCompletion(MapBuilder assembler, Completion context) { + super(assembler, context); + } + + @Override + protected Schema outerSchema(Schema inner) { + return Schema.createMap(inner); + } + } + + private static class ArrayCompletion extends NestedCompletion { + private ArrayCompletion(ArrayBuilder assembler, Completion context) { + super(assembler, context); + } + + @Override + protected Schema outerSchema(Schema inner) { + return Schema.createArray(inner); + } + } + + private static class UnionCompletion extends + Completion> { + private final Completion context; + private final NameContext names; + private final List schemas; + + private UnionCompletion(Completion context, NameContext names, List schemas) { + this.context = context; + this.names = names; + this.schemas = schemas; + } + + @Override + protected UnionAccumulator complete(Schema schema) { + List updated = new ArrayList(this.schemas); + updated.add(schema); + return new UnionAccumulator(context, names, updated); + } + } + + /** Accumulates all of the types in a union. Add an additional type with + * {@link #and()}. Complete the union with {@link #endUnion()} + */ + public static final class UnionAccumulator { + private final Completion context; + private final NameContext names; + private final List schemas; + + private UnionAccumulator(Completion context, NameContext names, List schemas) { + this.context = context; + this.names = names; + this.schemas = schemas; + } + + /** Add an additional type to this union **/ + public BaseTypeBuilder> and() { + return new UnionBuilder(context, names, schemas); + } + + /** Complete this union **/ + public R endUnion() { + Schema schema = Schema.createUnion(schemas); + return context.complete(schema); + } + } + + private static void checkRequired(Object reference, String errorMessage) { + if (reference == null) { + throw new NullPointerException(errorMessage); + } + } +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/SchemaParseException.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/SchemaParseException.java new file mode 100644 index 00000000000..09827914d44 --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/SchemaParseException.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +/** Thrown for errors parsing schemas and protocols. */ +public class SchemaParseException extends DataRuntimeException { + public SchemaParseException(Throwable cause) { + super(cause); + } + + public SchemaParseException(String message) { + super(message); + } +} + diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/UnresolvedUnionException.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/UnresolvedUnionException.java new file mode 100644 index 00000000000..6f7f5ce10ee --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/UnresolvedUnionException.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +/** Thrown when the expected contents of a union cannot be resolved. */ +public class UnresolvedUnionException extends DataRuntimeException { + private Object unresolvedDatum; + private Schema unionSchema; + + public UnresolvedUnionException(Schema unionSchema, Object unresolvedDatum) { + super("Not in union " + unionSchema + ": " + unresolvedDatum); + this.unionSchema = unionSchema; + this.unresolvedDatum = unresolvedDatum; + } + + public Object getUnresolvedDatum() { + return unresolvedDatum; + } + + public Schema getUnionSchema() { + return unionSchema; + } +} diff --git a/copycat-data/src/main/java/org/apache/kafka/copycat/data/Utf8.java b/copycat-data/src/main/java/org/apache/kafka/copycat/data/Utf8.java new file mode 100644 index 00000000000..e7a88649bd3 --- /dev/null +++ b/copycat-data/src/main/java/org/apache/kafka/copycat/data/Utf8.java @@ -0,0 +1,158 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +import java.nio.charset.Charset; + +/** A Utf8 string. Unlike {@link String}, instances are mutable. This is more + * efficient than {@link String} when reading or writing a sequence of values, + * as a single instance may be reused. */ +public class Utf8 implements Comparable, CharSequence { + private static final byte[] EMPTY = new byte[0]; + private static final Charset UTF8 = Charset.forName("UTF-8"); + + private byte[] bytes = EMPTY; + private int length; + private String string; + + public Utf8() { + } + + public Utf8(String string) { + this.bytes = getBytesFor(string); + this.length = bytes.length; + this.string = string; + } + + public Utf8(Utf8 other) { + this.length = other.length; + this.bytes = new byte[other.length]; + System.arraycopy(other.bytes, 0, this.bytes, 0, this.length); + this.string = other.string; + } + + public Utf8(byte[] bytes) { + this.bytes = bytes; + this.length = bytes.length; + } + + /** Return UTF-8 encoded bytes. + * Only valid through {@link #getByteLength()}. */ + public byte[] getBytes() { + return bytes; + } + + /** Return length in bytes. + * @deprecated call {@link #getByteLength()} instead. */ + public int getLength() { + return length; + } + + /** Return length in bytes. */ + public int getByteLength() { + return length; + } + + /** Set length in bytes. Should called whenever byte content changes, even + * if the length does not change, as this also clears the cached String. + * @deprecated call {@link #setByteLength(int)} instead. */ + public Utf8 setLength(int newLength) { + return setByteLength(newLength); + } + + /** Set length in bytes. Should called whenever byte content changes, even + * if the length does not change, as this also clears the cached String. */ + public Utf8 setByteLength(int newLength) { + if (this.bytes.length < newLength) { + byte[] newBytes = new byte[newLength]; + System.arraycopy(bytes, 0, newBytes, 0, this.length); + this.bytes = newBytes; + } + this.length = newLength; + this.string = null; + return this; + } + + /** Set to the contents of a String. */ + public Utf8 set(String string) { + this.bytes = getBytesFor(string); + this.length = bytes.length; + this.string = string; + return this; + } + + @Override + public String toString() { + if (this.length == 0) return ""; + if (this.string == null) { + this.string = new String(bytes, 0, length, UTF8); + } + return this.string; + } + + @Override + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof Utf8)) return false; + Utf8 that = (Utf8) o; + if (!(this.length == that.length)) return false; + byte[] thatBytes = that.bytes; + for (int i = 0; i < this.length; i++) + if (bytes[i] != thatBytes[i]) + return false; + return true; + } + + @Override + public int hashCode() { + int hash = 0; + for (int i = 0; i < this.length; i++) + hash = hash * 31 + bytes[i]; + return hash; + } + + @Override + public int compareTo(Utf8 that) { + return BinaryData.compareBytes(this.bytes, 0, this.length, + that.bytes, 0, that.length); + } + + // CharSequence implementation + @Override + public char charAt(int index) { + return toString().charAt(index); + } + + @Override + public int length() { + return toString().length(); + } + + @Override + public CharSequence subSequence(int start, int end) { + return toString().subSequence(start, end); + } + + /** Gets the UTF-8 bytes for a String */ + public static final byte[] getBytesFor(String str) { + return str.getBytes(UTF8); + } + +} diff --git a/copycat-data/src/test/java/org/apache/kafka/copycat/data/TestGenericData.java b/copycat-data/src/test/java/org/apache/kafka/copycat/data/TestGenericData.java new file mode 100644 index 00000000000..f064efbc7e4 --- /dev/null +++ b/copycat-data/src/test/java/org/apache/kafka/copycat/data/TestGenericData.java @@ -0,0 +1,410 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ +package org.apache.kafka.copycat.data; + +import org.apache.kafka.copycat.data.GenericData.Record; +import org.apache.kafka.copycat.data.Schema.Field; +import org.apache.kafka.copycat.data.Schema.Type; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.*; + +import static org.junit.Assert.*; + +public class TestGenericData { + + @Test(expected = DataRuntimeException.class) + public void testrecordConstructorNullSchema() throws Exception { + new GenericData.Record(null); + } + + @Test(expected = DataRuntimeException.class) + public void testrecordConstructorWrongSchema() throws Exception { + new GenericData.Record(Schema.create(Schema.Type.INT)); + } + + @Test(expected = DataRuntimeException.class) + public void testArrayConstructorNullSchema() throws Exception { + new GenericData.Array(1, null); + } + + @Test(expected = DataRuntimeException.class) + public void testArrayConstructorWrongSchema() throws Exception { + new GenericData.Array(1, Schema.create(Schema.Type.INT)); + } + + @Test(expected = DataRuntimeException.class) + public void testRecordCreateEmptySchema() throws Exception { + Schema s = Schema.createRecord("schemaName", "schemaDoc", "namespace", false); + Record r = new GenericData.Record(s); + } + + @Test(expected = DataRuntimeException.class) + public void testGetEmptySchemaFields() throws Exception { + Schema s = Schema.createRecord("schemaName", "schemaDoc", "namespace", false); + s.getFields(); + } + + @Test(expected = DataRuntimeException.class) + public void testGetEmptySchemaField() throws Exception { + Schema s = Schema.createRecord("schemaName", "schemaDoc", "namespace", false); + s.getField("foo"); + } + + @Test(expected = DataRuntimeException.class) + public void testRecordPutInvalidField() throws Exception { + Schema s = Schema.createRecord("schemaName", "schemaDoc", "namespace", false); + List fields = new ArrayList(); + fields.add(new Schema.Field("someFieldName", s, "docs", null)); + s.setFields(fields); + Record r = new GenericData.Record(s); + r.put("invalidFieldName", "someValue"); + } + + @Test + /** Make sure that even with nulls, hashCode() doesn't throw NPE. */ + public void testHashCode() { + GenericData.get().hashCode(null, Schema.create(Type.NULL)); + GenericData.get().hashCode(null, Schema.createUnion( + Arrays.asList(Schema.create(Type.BOOLEAN), Schema.create(Type.STRING)))); + List stuff = new ArrayList(); + stuff.add("string"); + Schema schema = recordSchema(); + GenericRecord r = new GenericData.Record(schema); + r.put(0, stuff); + GenericData.get().hashCode(r, schema); + } + + @Test + public void testEquals() { + Schema s = recordSchema(); + GenericRecord r0 = new GenericData.Record(s); + GenericRecord r1 = new GenericData.Record(s); + GenericRecord r2 = new GenericData.Record(s); + Collection l0 = new ArrayDeque(); + List l1 = new ArrayList(); + GenericArray l2 = + new GenericData.Array(1, s.getFields().get(0).schema()); + String foo = "foo"; + l0.add(new StringBuffer(foo)); + l1.add(foo); + l2.add(new Utf8(foo)); + r0.put(0, l0); + r1.put(0, l1); + r2.put(0, l2); + assertEquals(r0, r1); + assertEquals(r0, r2); + assertEquals(r1, r2); + } + + private Schema recordSchema() { + List fields = new ArrayList(); + fields.add(new Field("anArray", Schema.createArray(Schema.create(Type.STRING)), null, null)); + Schema schema = Schema.createRecord("arrayFoo", "test", "mytest", false); + schema.setFields(fields); + + return schema; + } + + @Test + public void testEquals2() { + Schema schema1 = Schema.createRecord("r", null, "x", false); + List fields1 = new ArrayList(); + fields1.add(new Field("a", Schema.create(Schema.Type.STRING), null, null, + Field.Order.IGNORE)); + schema1.setFields(fields1); + + // only differs in field order + Schema schema2 = Schema.createRecord("r", null, "x", false); + List fields2 = new ArrayList(); + fields2.add(new Field("a", Schema.create(Schema.Type.STRING), null, null, + Field.Order.ASCENDING)); + schema2.setFields(fields2); + + GenericRecord record1 = new GenericData.Record(schema1); + record1.put("a", "1"); + + GenericRecord record2 = new GenericData.Record(schema2); + record2.put("a", "2"); + + assertFalse(record2.equals(record1)); + assertFalse(record1.equals(record2)); + } + + @Test + public void testRecordGetFieldDoesntExist() throws Exception { + List fields = new ArrayList(); + Schema schema = Schema.createRecord(fields); + GenericData.Record record = new GenericData.Record(schema); + assertNull(record.get("does not exist")); + } + + @Test + public void testArrayReversal() { + Schema schema = Schema.createArray(Schema.create(Schema.Type.INT)); + GenericArray forward = new GenericData.Array(10, schema); + GenericArray backward = new GenericData.Array(10, schema); + for (int i = 0; i <= 9; i++) { + forward.add(i); + } + for (int i = 9; i >= 0; i--) { + backward.add(i); + } + forward.reverse(); + assertTrue(forward.equals(backward)); + } + + @Test + public void testArrayListInterface() { + Schema schema = Schema.createArray(Schema.create(Schema.Type.INT)); + GenericArray array = new GenericData.Array(1, schema); + array.add(99); + assertEquals(new Integer(99), array.get(0)); + List list = new ArrayList(); + list.add(99); + assertEquals(array, list); + assertEquals(list, array); + assertEquals(list.hashCode(), array.hashCode()); + try { + array.get(2); + fail("Expected IndexOutOfBoundsException getting index 2"); + } catch (IndexOutOfBoundsException e) { + } + array.clear(); + assertEquals(0, array.size()); + try { + array.get(0); + fail("Expected IndexOutOfBoundsException getting index 0 after clear()"); + } catch (IndexOutOfBoundsException e) { + } + + } + + @Test + public void testArrayAddAtLocation() { + Schema schema = Schema.createArray(Schema.create(Schema.Type.INT)); + GenericArray array = new GenericData.Array(6, schema); + array.clear(); + for (int i = 0; i < 5; ++i) + array.add(i); + assertEquals(5, array.size()); + array.add(0, 6); + assertEquals(new Integer(6), array.get(0)); + assertEquals(6, array.size()); + assertEquals(new Integer(0), array.get(1)); + assertEquals(new Integer(4), array.get(5)); + array.add(6, 7); + assertEquals(new Integer(7), array.get(6)); + assertEquals(7, array.size()); + assertEquals(new Integer(6), array.get(0)); + assertEquals(new Integer(4), array.get(5)); + array.add(1, 8); + assertEquals(new Integer(8), array.get(1)); + assertEquals(new Integer(0), array.get(2)); + assertEquals(new Integer(6), array.get(0)); + assertEquals(8, array.size()); + try { + array.get(9); + fail("Expected IndexOutOfBoundsException after adding elements"); + } catch (IndexOutOfBoundsException e) { + } + } + + @Test + public void testArrayRemove() { + Schema schema = Schema.createArray(Schema.create(Schema.Type.INT)); + GenericArray array = new GenericData.Array(10, schema); + array.clear(); + for (int i = 0; i < 10; ++i) + array.add(i); + assertEquals(10, array.size()); + assertEquals(new Integer(0), array.get(0)); + assertEquals(new Integer(9), array.get(9)); + + array.remove(0); + assertEquals(9, array.size()); + assertEquals(new Integer(1), array.get(0)); + assertEquals(new Integer(2), array.get(1)); + assertEquals(new Integer(9), array.get(8)); + + // Test boundary errors. + try { + array.get(9); + fail("Expected IndexOutOfBoundsException after removing an element"); + } catch (IndexOutOfBoundsException e) { + } + try { + array.set(9, 99); + fail("Expected IndexOutOfBoundsException after removing an element"); + } catch (IndexOutOfBoundsException e) { + } + try { + array.remove(9); + fail("Expected IndexOutOfBoundsException after removing an element"); + } catch (IndexOutOfBoundsException e) { + } + + // Test that we can still remove for properly sized arrays, and the rval + assertEquals(new Integer(9), array.remove(8)); + assertEquals(8, array.size()); + + + // Test insertion after remove + array.add(88); + assertEquals(new Integer(88), array.get(8)); + } + + @Test + public void testArraySet() { + Schema schema = Schema.createArray(Schema.create(Schema.Type.INT)); + GenericArray array = new GenericData.Array(10, schema); + array.clear(); + for (int i = 0; i < 10; ++i) + array.add(i); + assertEquals(10, array.size()); + assertEquals(new Integer(0), array.get(0)); + assertEquals(new Integer(5), array.get(5)); + + assertEquals(new Integer(5), array.set(5, 55)); + assertEquals(10, array.size()); + assertEquals(new Integer(55), array.get(5)); + } + + @Test + public void testToStringDoesNotEscapeForwardSlash() throws Exception { + GenericData data = GenericData.get(); + assertEquals("\"/\"", data.toString("/")); + } + + @Test + public void testToStringNanInfinity() throws Exception { + GenericData data = GenericData.get(); + assertEquals("\"Infinity\"", data.toString(Float.POSITIVE_INFINITY)); + assertEquals("\"-Infinity\"", data.toString(Float.NEGATIVE_INFINITY)); + assertEquals("\"NaN\"", data.toString(Float.NaN)); + assertEquals("\"Infinity\"", data.toString(Double.POSITIVE_INFINITY)); + assertEquals("\"-Infinity\"", data.toString(Double.NEGATIVE_INFINITY)); + assertEquals("\"NaN\"", data.toString(Double.NaN)); + } + + @Test + public void testEnumCompare() { + Schema s = Schema.createEnum("Kind", null, null, Arrays.asList("Z", "Y", "X")); + GenericEnumSymbol z = new GenericData.EnumSymbol(s, "Z"); + GenericEnumSymbol y = new GenericData.EnumSymbol(s, "Y"); + assertEquals(0, z.compareTo(z)); + assertTrue(y.compareTo(z) > 0); + assertTrue(z.compareTo(y) < 0); + } + + @Test + public void testByteBufferDeepCopy() { + // Test that a deep copy of a byte buffer respects the byte buffer + // limits and capacity. + byte[] buffer_value = {0, 1, 2, 3, 0, 0, 0}; + ByteBuffer buffer = ByteBuffer.wrap(buffer_value, 1, 4); + Schema schema = Schema.createRecord("my_record", "doc", "mytest", false); + Field byte_field = + new Field("bytes", Schema.create(Type.BYTES), null, null); + schema.setFields(Arrays.asList(byte_field)); + + GenericRecord record = new GenericData.Record(schema); + record.put(byte_field.name(), buffer); + + GenericRecord copy = GenericData.get().deepCopy(schema, record); + ByteBuffer buffer_copy = (ByteBuffer) copy.get(byte_field.name()); + + assertEquals(buffer, buffer_copy); + } + + @Test + public void testValidateNullableEnum() { + List unionTypes = new ArrayList(); + Schema schema; + Schema nullSchema = Schema.create(Type.NULL); + Schema enumSchema = Schema.createEnum("AnEnum", null, null, Arrays.asList("X", "Y", "Z")); + GenericEnumSymbol w = new GenericData.EnumSymbol(enumSchema, "W"); + GenericEnumSymbol x = new GenericData.EnumSymbol(enumSchema, "X"); + GenericEnumSymbol y = new GenericData.EnumSymbol(enumSchema, "Y"); + GenericEnumSymbol z = new GenericData.EnumSymbol(enumSchema, "Z"); + + // null is first + unionTypes.clear(); + unionTypes.add(nullSchema); + unionTypes.add(enumSchema); + schema = Schema.createUnion(unionTypes); + + assertTrue(GenericData.get().validate(schema, z)); + assertTrue(GenericData.get().validate(schema, y)); + assertTrue(GenericData.get().validate(schema, x)); + assertFalse(GenericData.get().validate(schema, w)); + assertTrue(GenericData.get().validate(schema, null)); + + // null is last + unionTypes.clear(); + unionTypes.add(enumSchema); + unionTypes.add(nullSchema); + schema = Schema.createUnion(unionTypes); + + assertTrue(GenericData.get().validate(schema, z)); + assertTrue(GenericData.get().validate(schema, y)); + assertTrue(GenericData.get().validate(schema, x)); + assertFalse(GenericData.get().validate(schema, w)); + assertTrue(GenericData.get().validate(schema, null)); + } + + private enum anEnum {ONE, TWO, THREE} + + ; + + @Test + public void validateRequiresGenericSymbolForEnumSchema() { + final Schema schema = Schema.createEnum("my_enum", "doc", "namespace", Arrays.asList("ONE", "TWO", "THREE")); + final GenericData gd = GenericData.get(); + + /* positive cases */ + assertTrue(gd.validate(schema, new GenericData.EnumSymbol(schema, "ONE"))); + assertTrue(gd.validate(schema, new GenericData.EnumSymbol(schema, anEnum.ONE))); + + /* negative cases */ + assertFalse("We don't expect GenericData to allow a String datum for an enum schema", gd.validate(schema, "ONE")); + assertFalse("We don't expect GenericData to allow a Java Enum for an enum schema", gd.validate(schema, anEnum.ONE)); + } + + @Test + public void testValidateUnion() { + Schema type1Schema = SchemaBuilder.record("Type1") + .fields() + .requiredString("myString") + .requiredInt("myInt") + .endRecord(); + + Schema type2Schema = SchemaBuilder.record("Type2") + .fields() + .requiredString("myString") + .endRecord(); + + Schema unionSchema = SchemaBuilder.unionOf() + .type(type1Schema).and().type(type2Schema) + .endUnion(); + + GenericRecord record = new GenericData.Record(type2Schema); + record.put("myString", "myValue"); + assertTrue(GenericData.get().validate(unionSchema, record)); + } +} diff --git a/settings.gradle b/settings.gradle index 3b6a952cff2..97416a573b7 100644 --- a/settings.gradle +++ b/settings.gradle @@ -14,4 +14,5 @@ // limitations under the License. apply from: file('scala.gradle') -include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'log4j-appender' +include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', + 'log4j-appender', 'copycat-data', 'copycat-api'