diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml
index 13aa1d7ca..b811a2118 100644
--- a/hoodie-client/pom.xml
+++ b/hoodie-client/pom.xml
@@ -64,6 +64,45 @@
hoodie-common
${project.version}
+
+ org.apache.hadoop
+ hadoop-hdfs
+ tests
+
+
+
+ org.mortbay.jetty
+ *
+
+
+ javax.servlet.jsp
+ *
+
+
+ javax.servlet
+ *
+
+
+
+
+ org.apache.hadoop
+ hadoop-common
+ tests
+
+
+ org.mortbay.jetty
+ *
+
+
+ javax.servlet.jsp
+ *
+
+
+ javax.servlet
+ *
+
+
+
com.uber.hoodie
hoodie-common
diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml
index 7c81674b6..bb1929813 100644
--- a/hoodie-common/pom.xml
+++ b/hoodie-common/pom.xml
@@ -90,6 +90,21 @@
1.10.19
test
+
+ org.apache.hadoop
+ hadoop-hdfs
+ tests
+
+
+ org.apache.hadoop
+ hadoop-common
+ tests
+
+
+ org.codehaus.jackson
+ jackson-core-asl
+ 1.9.13
+
org.apache.commons
commons-lang3
@@ -99,5 +114,15 @@
kryo
test
+
+
+
+
+
+
+
+
+
+
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogAppendConfig.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogAppendConfig.java
new file mode 100644
index 000000000..34d727ede
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogAppendConfig.java
@@ -0,0 +1,228 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.common.table.log;
+
+import com.uber.hoodie.common.util.FSUtils;
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+
+/**
+ * Configuration for a HoodieLog
+ */
+public class HoodieLogAppendConfig {
+ private final static Logger log = LogManager.getLogger(HoodieLogAppendConfig.class);
+ private static final long DEFAULT_SIZE_THRESHOLD = 32 * 1024 * 1024L;
+
+ private final int bufferSize;
+ private final short replication;
+ private final long blockSize;
+ private final HoodieLogFile logFile;
+ private boolean isAutoFlush;
+ private final Schema schema;
+ private final FileSystem fs;
+ private final long sizeThreshold;
+
+ private HoodieLogAppendConfig(FileSystem fs, HoodieLogFile logFile, Schema schema, Integer bufferSize,
+ Short replication, Long blockSize, boolean isAutoFlush, Long sizeThreshold) {
+ this.fs = fs;
+ this.logFile = logFile;
+ this.schema = schema;
+ this.bufferSize = bufferSize;
+ this.replication = replication;
+ this.blockSize = blockSize;
+ this.isAutoFlush = isAutoFlush;
+ this.sizeThreshold = sizeThreshold;
+ }
+
+ public int getBufferSize() {
+ return bufferSize;
+ }
+
+ public short getReplication() {
+ return replication;
+ }
+
+ public long getBlockSize() {
+ return blockSize;
+ }
+
+ public Schema getSchema() {
+ return schema;
+ }
+
+ public FileSystem getFs() {
+ return fs;
+ }
+
+ public HoodieLogFile getLogFile() {
+ return logFile;
+ }
+
+ public long getSizeThreshold() {
+ return sizeThreshold;
+ }
+
+ public boolean isAutoFlush() {
+ return isAutoFlush;
+ }
+
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ public HoodieLogAppendConfig withLogFile(HoodieLogFile newFile) {
+ return new HoodieLogAppendConfig(fs, newFile, schema, bufferSize, replication, blockSize,
+ isAutoFlush, sizeThreshold);
+ }
+
+ public static class Builder {
+ // Auto-flush. if set to true - then after every append, the avro block will be flushed
+ private boolean isAutoFlush = true;
+ // Buffer size in the Avro writer
+ private Integer bufferSize;
+ // Replication for the log file
+ private Short replication;
+ // Blocksize for the avro log file (useful if auto-flush is set to false)
+ private Long blockSize;
+ // Schema for the log file
+ private Schema schema;
+ // FileSystem
+ private FileSystem fs;
+ // Size threshold for the log file. Useful when used with a rolling log appender
+ private Long sizeThreshold;
+ // Log File extension. Could be .avro.delta or .avro.commits etc
+ private String logFileExtension;
+ // File ID
+ private String fileId;
+ // version number for this log file. If not specified, then the current version will be computed
+ private Integer fileVersion;
+ // Partition path for the log file
+ private Path partitionPath;
+
+ public Builder withBufferSize(int bufferSize) {
+ this.bufferSize = bufferSize;
+ return this;
+ }
+
+ public Builder withReplication(short replication) {
+ this.replication = replication;
+ return this;
+ }
+
+ public Builder withBlockSize(long blockSize) {
+ this.blockSize = blockSize;
+ return this;
+ }
+
+ public Builder withSchema(Schema schema) {
+ this.schema = schema;
+ return this;
+ }
+
+ public Builder withFs(FileSystem fs) {
+ this.fs = fs;
+ return this;
+ }
+
+ public Builder withAutoFlush(boolean autoFlush) {
+ this.isAutoFlush = autoFlush;
+ return this;
+ }
+
+ public Builder withSizeThreshold(long sizeThreshold) {
+ this.sizeThreshold = sizeThreshold;
+ return this;
+ }
+
+ public Builder withLogFileExtension(String logFileExtension) {
+ this.logFileExtension = logFileExtension;
+ return this;
+ }
+
+ public Builder withFileId(String fileId) {
+ this.fileId = fileId;
+ return this;
+ }
+
+ public Builder withFileVersion(int version) {
+ this.fileVersion = version;
+ return this;
+ }
+
+ public Builder onPartitionPath(Path path) {
+ this.partitionPath = path;
+ return this;
+ }
+
+ public HoodieLogAppendConfig build() throws IOException {
+ log.info("Building HoodieLogAppendConfig");
+ if (schema == null) {
+ throw new IllegalArgumentException("Schema for log is not specified");
+ }
+ if (fs == null) {
+ fs = FSUtils.getFs();
+ }
+
+ if (fileId == null) {
+ throw new IllegalArgumentException("FileID is not specified");
+ }
+ if (logFileExtension == null) {
+ throw new IllegalArgumentException("File extension is not specified");
+ }
+ if (partitionPath == null) {
+ throw new IllegalArgumentException("Partition path is not specified");
+ }
+ if (fileVersion == null) {
+ log.info("Computing the next log version for " + fileId + " in " + partitionPath);
+ fileVersion =
+ FSUtils.getCurrentLogVersion(fs, partitionPath, fileId, logFileExtension);
+ log.info(
+ "Computed the next log version for " + fileId + " in " + partitionPath + " as "
+ + fileVersion);
+ }
+
+ Path logPath = new Path(partitionPath,
+ FSUtils.makeLogFileName(fileId, logFileExtension, fileVersion));
+ log.info("LogConfig created on path " + logPath);
+ HoodieLogFile logFile = new HoodieLogFile(logPath);
+
+ if (bufferSize == null) {
+ bufferSize = FSUtils.getDefaultBufferSize(fs);
+ }
+ if (replication == null) {
+ replication = FSUtils.getDefaultReplication(fs, partitionPath);
+ }
+ if (blockSize == null) {
+ blockSize = FSUtils.getDefaultBlockSize(fs, partitionPath);
+ }
+ if (sizeThreshold == null) {
+ sizeThreshold = DEFAULT_SIZE_THRESHOLD;
+ }
+
+ return new HoodieLogAppendConfig(fs, logFile, schema, bufferSize, replication, blockSize,
+ isAutoFlush, sizeThreshold);
+
+ }
+
+
+ }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogAppender.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogAppender.java
new file mode 100644
index 000000000..afa6a3908
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogAppender.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.common.table.log;
+
+import com.uber.hoodie.common.table.log.avro.AvroLogAppender;
+import com.uber.hoodie.common.table.log.avro.RollingAvroLogAppender;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Interface for implementations supporting appending data to a log file
+ *
+ * @param
+ * @see AvroLogAppender
+ * @see RollingAvroLogAppender
+ */
+public interface HoodieLogAppender {
+ /**
+ * Append a stream of records in a batch (this will be written as a block/unit to the underlying log)
+ *
+ * @param records
+ * @throws IOException
+ */
+ void append(List records) throws IOException, InterruptedException;
+
+ /**
+ * Syncs the log manually if auto-flush is not set in HoodieLogAppendConfig. If auto-flush is set
+ * Then the LogAppender will automatically flush after the append call.
+ *
+ * @throws IOException
+ */
+ void sync() throws IOException;
+
+ /**
+ * Close the appended and release any resources holding on to
+ *
+ * @throws IOException
+ */
+ void close() throws IOException;
+
+ /**
+ * Gets the current offset in the log. This is usually used to mark the start of the block in
+ * meta-data and passed to the HoodieLogReader
+ *
+ * @return
+ * @throws IOException
+ */
+ long getCurrentSize() throws IOException;
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFile.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFile.java
new file mode 100644
index 000000000..356cfbf17
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFile.java
@@ -0,0 +1,92 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.common.table.log;
+
+import com.uber.hoodie.common.util.FSUtils;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Optional;
+
+/**
+ * Abstracts a single log file. Contains methods to extract metadata like the fileId, version and
+ * extension from the log file path.
+ *
+ * Also contains logic to roll-over the log file
+ */
+public class HoodieLogFile {
+ public static final String DELTA_EXTENSION = ".avro.delta";
+
+ private final Path path;
+ private Optional fileStatus;
+
+ public HoodieLogFile(FileStatus fileStatus) {
+ this(fileStatus.getPath());
+ this.fileStatus = Optional.of(fileStatus);
+ }
+
+ public HoodieLogFile(Path logPath) {
+ this.path = logPath;
+ this.fileStatus = Optional.empty();
+ }
+
+ public String getFileId() {
+ return FSUtils.getFileIdFromLogPath(path);
+ }
+
+ public int getLogVersion() {
+ return FSUtils.getFileVersionFromLog(path);
+ }
+
+ public String getFileExtension() {
+ return FSUtils.getFileExtensionFromLog(path);
+ }
+
+ public Path getPath() {
+ return path;
+ }
+
+ public String getFileName() {
+ return path.getName();
+ }
+
+ public Optional getFileStatus() {
+ return fileStatus;
+ }
+
+
+ public HoodieLogFile rollOver(FileSystem fs) throws IOException {
+ String fileId = getFileId();
+ int newVersion =
+ FSUtils.computeNextLogVersion(fs, path.getParent(), fileId, DELTA_EXTENSION);
+ return new HoodieLogFile(new Path(path.getParent(),
+ FSUtils.makeLogFileName(fileId, DELTA_EXTENSION, newVersion)));
+ }
+
+ public boolean shouldRollOver(HoodieLogAppender currentWriter, HoodieLogAppendConfig config)
+ throws IOException {
+ return currentWriter.getCurrentSize() > config.getSizeThreshold();
+ }
+
+ @Override
+ public String toString() {
+ return "HoodieLogFile{" + path + '}';
+ }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/avro/AvroLogAppender.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/avro/AvroLogAppender.java
new file mode 100644
index 000000000..ef29ac8ef
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/avro/AvroLogAppender.java
@@ -0,0 +1,140 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.common.table.log.avro;
+
+import com.uber.hoodie.common.table.log.HoodieLogAppendConfig;
+import com.uber.hoodie.common.table.log.HoodieLogAppender;
+import com.uber.hoodie.common.util.FSUtils;
+import com.uber.hoodie.exception.HoodieException;
+import com.uber.hoodie.exception.HoodieIOException;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.fs.AvroFSInput;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * AvroLogAppender appends a bunch of IndexedRecord to a Avro data file.
+ * If auto-flush is set, every call to append writes out a block.
+ * A avro block corresponds to records appended in a single commit.
+ *
+ * @see org.apache.avro.file.DataFileReader
+ */
+public class AvroLogAppender implements HoodieLogAppender {
+ private final static Logger log = LogManager.getLogger(AvroLogAppender.class);
+ private final HoodieLogAppendConfig config;
+ private FSDataOutputStream output;
+ private DataFileWriter writer;
+ private boolean autoFlush;
+
+ public AvroLogAppender(HoodieLogAppendConfig config) throws IOException, InterruptedException {
+ FileSystem fs = config.getFs();
+ this.config = config;
+ this.autoFlush = config.isAutoFlush();
+ GenericDatumWriter datumWriter =
+ new GenericDatumWriter<>(config.getSchema());
+ this.writer = new DataFileWriter<>(datumWriter);
+ Path path = config.getLogFile().getPath();
+
+ if (fs.exists(path)) {
+ //TODO - check for log corruption and roll over if needed
+ log.info(config.getLogFile() + " exists. Appending to existing file");
+ // this log path exists, we will append to it
+ fs = FileSystem.get(fs.getConf());
+ try {
+ this.output = fs.append(path, config.getBufferSize());
+ } catch (RemoteException e) {
+ // this happens when either another task executor writing to this file died or data node is going down
+ if (e.getClassName().equals(AlreadyBeingCreatedException.class.getName())
+ && fs instanceof DistributedFileSystem) {
+ log.warn("Trying to recover log on path " + path);
+ if (FSUtils.recoverDFSFileLease((DistributedFileSystem) fs, path)) {
+ log.warn("Recovered lease on path " + path);
+ // try again
+ this.output = fs.append(path, config.getBufferSize());
+ } else {
+ log.warn("Failed to recover lease on path " + path);
+ throw new HoodieException(e);
+ }
+ }
+ }
+ this.writer
+ .appendTo(new AvroFSInput(FileContext.getFileContext(fs.getConf()), path), output);
+ // we always want to flush to disk everytime a avro block is written
+ this.writer.setFlushOnEveryBlock(true);
+ } else {
+ log.info(config.getLogFile() + " does not exist. Create a new file");
+ this.output = fs.create(path, false, config.getBufferSize(), config.getReplication(),
+ config.getBlockSize(), null);
+ this.writer.create(config.getSchema(), output);
+ this.writer.setFlushOnEveryBlock(true);
+ // We need to close the writer to be able to tell the name node that we created this file
+ // this.writer.close();
+ }
+ }
+
+ public void append(List records) throws IOException {
+ records.forEach(r -> {
+ try {
+ writer.append(r);
+ } catch (IOException e) {
+ throw new HoodieIOException(
+ "Could not append record " + r + " to " + config.getLogFile());
+ }
+ });
+ if (autoFlush) {
+ sync();
+ }
+ }
+
+ public void sync() throws IOException {
+ if (output == null || writer == null)
+ return; // Presume closed
+ writer.flush();
+ output.flush();
+ output.hflush();
+ }
+
+ public void close() throws IOException {
+ sync();
+ writer.close();
+ writer = null;
+ output.close();
+ output = null;
+ }
+
+ public long getCurrentSize() throws IOException {
+ if (writer == null) {
+ throw new IllegalStateException(
+ "LogWriter " + config.getLogFile() + " has been closed. Cannot getCurrentSize");
+ }
+ // writer.sync() returns only the offset for this block and not the global offset
+ return output.getPos();
+ }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/avro/AvroLogReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/avro/AvroLogReader.java
new file mode 100644
index 000000000..4f9a1a8d5
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/avro/AvroLogReader.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.common.table.log.avro;
+
+import com.google.common.collect.Lists;
+import com.uber.hoodie.common.table.log.HoodieLogAppender;
+import com.uber.hoodie.common.table.log.HoodieLogFile;
+import com.uber.hoodie.exception.HoodieIOException;
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.AvroFSInput;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileSystem;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * AvroLogReader allows reading blocks of records given a offset as written by AvroLogAppender
+ * Avro Log files are never streamed entirely - because of fault tolerance.
+ * If a block is corrupted, then random access with offset bypasses any corrupt blocks.
+ * Metadata about offset should be saved when writing blocks and passed in readBlock()
+ *
+ * @see AvroLogAppender
+ */
+public class AvroLogReader {
+ private final DataFileReader reader;
+ private final HoodieLogFile file;
+
+ public AvroLogReader(HoodieLogFile file, FileSystem fs, Schema readerSchema)
+ throws IOException {
+ GenericDatumReader datumReader = new GenericDatumReader<>();
+ datumReader.setExpected(readerSchema);
+ final AvroFSInput input = new AvroFSInput(FileContext.getFileContext(fs.getConf()), file.getPath());
+ this.reader = (DataFileReader) DataFileReader.openReader(input, datumReader);
+ this.file = file;
+ }
+
+ public Iterator readBlock(long startOffset) throws IOException {
+ // We keep track of exact offset for blocks, just seek to it directly
+ reader.seek(startOffset);
+
+ List records = Lists.newArrayList();
+ try {
+ // First check if we are past the sync market and then check reader.hasNext,
+ // hasNext will load a block in memory and this will fail if a block is corrupted.
+ while (!reader.pastSync(startOffset) && reader.hasNext()) {
+ records.add(reader.next());
+ }
+ } catch (IOException e) {
+ throw new HoodieIOException("Failed to read avro records from " + file);
+ }
+ return records.iterator();
+ }
+
+ public HoodieLogFile getFile() {
+ return file;
+ }
+
+ public void close() throws IOException {
+ reader.close();
+ }
+
+
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/avro/CompositeAvroLogReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/avro/CompositeAvroLogReader.java
new file mode 100644
index 000000000..7ed6f497d
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/avro/CompositeAvroLogReader.java
@@ -0,0 +1,155 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.common.table.log.avro;
+
+import com.uber.hoodie.common.table.log.HoodieLogFile;
+import com.uber.hoodie.common.util.FSUtils;
+import com.uber.hoodie.exception.HoodieIOException;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * CompositeAvroLogReader reads all versions of the logs for a given fileId.
+ * It gives a iterator which iterates through all the versions and the list of blocks for that specific version
+ * Useful for merging records in RecordReader and compacting all the delta versions
+ *
+ * @see AvroLogReader
+ */
+public class CompositeAvroLogReader {
+ private final Map readers;
+
+ public CompositeAvroLogReader(Path partitionPath, String fileId, FileSystem fs,
+ Schema readerSchema, String logFileExtension) throws IOException {
+ Stream allLogFiles =
+ FSUtils.getAllLogFiles(fs, partitionPath, fileId, logFileExtension);
+ this.readers = allLogFiles.map(hoodieLogFile -> {
+ try {
+ return new AvroLogReader(hoodieLogFile, fs, readerSchema);
+ } catch (IOException e) {
+ throw new HoodieIOException(
+ "Could not read avro records from path " + hoodieLogFile);
+ }
+ }).collect(Collectors.toMap(new Function() {
+ @Override
+ public Integer apply(AvroLogReader avroLogReader) {
+ return avroLogReader.getFile().getLogVersion();
+ }
+ }, Function.identity()));
+ }
+
+ /**
+ * Reads all the versions (in the order specified) and all the blocks starting with the offset specified
+ *
+ * @param filesToOffsetMap
+ * @return
+ * @throws IOException
+ */
+ public Iterator readBlocks(SortedMap> filesToOffsetMap)
+ throws IOException {
+ return new Iterators(filesToOffsetMap, readers);
+ }
+
+ public void close() throws IOException {
+ readers.values().forEach(s -> {
+ try {
+ s.close();
+ } catch (IOException e) {
+ throw new HoodieIOException("Unable to close " + s.getFile(), e);
+ }
+ });
+ }
+
+ public class Iterators implements Iterator {
+
+ private final Map readers;
+ private final Map> versionsToOffsetMap;
+ private Integer currentVersion;
+ private Iterator currentVersionIterator;
+ private Iterator currentOffsetIterator;
+ private Iterator currentRecordIterator;
+
+ public Iterators(Map> versionToOffsetMap,
+ Map readers) {
+ this.currentVersionIterator = versionToOffsetMap.keySet().iterator();
+ this.readers = readers;
+ this.versionsToOffsetMap = versionToOffsetMap;
+ }
+
+ private Iterator findNextBlock() throws IOException {
+ if (currentOffsetIterator != null) {
+ while (currentOffsetIterator.hasNext()) {
+ // we have more offsets to process for this file
+ long currentOffset = currentOffsetIterator.next();
+ Iterator currentBlock =
+ readers.get(currentVersion).readBlock(currentOffset);
+ if (currentBlock.hasNext()) {
+ return currentBlock;
+ }
+ }
+ }
+ return null;
+ }
+
+ private Iterator findNext() {
+ try {
+ Iterator nextBlock = findNextBlock();
+ if (nextBlock != null) {
+ // we have more offsets to process for this version
+ return nextBlock;
+ }
+
+ // We have no more offsets to process for the version, lets move on to the next version
+ while (currentVersionIterator.hasNext()) {
+ currentVersion = currentVersionIterator.next();
+ currentOffsetIterator = versionsToOffsetMap.get(currentVersion).iterator();
+ nextBlock = findNextBlock();
+ if (nextBlock != null) {
+ return nextBlock;
+ }
+ }
+ } catch (IOException e) {
+ throw new HoodieIOException(
+ "Could not read avro records from " + readers.get(currentVersion).getFile());
+ }
+ return null;
+ }
+
+ @Override
+ public boolean hasNext() {
+ if (currentRecordIterator == null || !currentRecordIterator.hasNext()) {
+ currentRecordIterator = findNext();
+ }
+ return (currentRecordIterator != null && currentRecordIterator.hasNext());
+ }
+
+ @Override
+ public GenericRecord next() {
+ return currentRecordIterator.next();
+ }
+ }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/avro/RollingAvroLogAppender.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/avro/RollingAvroLogAppender.java
new file mode 100644
index 000000000..ce82e8bba
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/avro/RollingAvroLogAppender.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.common.table.log.avro;
+
+import com.google.common.base.Preconditions;
+import com.uber.hoodie.common.table.log.HoodieLogAppendConfig;
+import com.uber.hoodie.common.table.log.HoodieLogAppender;
+import com.uber.hoodie.common.table.log.HoodieLogFile;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Implementation of {@link HoodieLogAppender} to roll over the log file when the sizeThreshold is reached.
+ */
+public class RollingAvroLogAppender implements HoodieLogAppender {
+ private static final Log LOG = LogFactory.getLog(RollingAvroLogAppender.class);
+ private AvroLogAppender logWriter;
+ private HoodieLogAppendConfig config;
+
+ public RollingAvroLogAppender(HoodieLogAppendConfig config)
+ throws IOException, InterruptedException {
+ // initialize
+ this.logWriter = new AvroLogAppender(config);
+ this.config = config;
+ rollOverIfNeeded();
+ }
+
+ private void rollOverIfNeeded() throws IOException, InterruptedException {
+ HoodieLogFile logFile = config.getLogFile();
+ boolean shouldRollOver = logFile.shouldRollOver(this, config);
+ if (shouldRollOver) {
+ if (logWriter != null) {
+ // Close the old writer and open a new one
+ logWriter.close();
+ }
+ // Current logWriter is not initialized, set the current file name
+ HoodieLogFile nextRollLogPath = logFile.rollOver(config.getFs());
+ LOG.info("Rolling over log from " + logFile + " to " + nextRollLogPath);
+ this.config = config.withLogFile(nextRollLogPath);
+ this.logWriter = new AvroLogAppender(this.config);
+ }
+ }
+
+ public long getCurrentSize() throws IOException {
+ Preconditions.checkArgument(logWriter != null);
+ return logWriter.getCurrentSize();
+ }
+
+ public void append(List records) throws IOException, InterruptedException {
+ LOG.info("Appending " + records.size() + " records to " + config.getLogFile());
+ rollOverIfNeeded();
+ Preconditions.checkArgument(logWriter != null);
+ logWriter.append(records);
+ }
+
+ public void sync() throws IOException {
+ Preconditions.checkArgument(logWriter != null);
+ logWriter.sync();
+ }
+
+ public void close() throws IOException {
+ Preconditions.checkArgument(logWriter != null);
+ logWriter.close();
+ }
+
+ public HoodieLogAppendConfig getConfig() {
+ return config;
+ }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java
index dc5b2c352..5cbc6f2ea 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java
@@ -17,21 +17,27 @@
package com.uber.hoodie.common.util;
import com.google.common.base.Preconditions;
-import com.uber.hoodie.common.table.HoodieTableMetaClient;
-import com.uber.hoodie.common.table.HoodieTimeline;
-import com.uber.hoodie.common.table.timeline.HoodieInstant;
+import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.exception.HoodieIOException;
+import com.uber.hoodie.exception.InvalidHoodiePathException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
import java.util.List;
+import java.util.Optional;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Stream;
/**
* Utility functions related to accessing the file storage
@@ -39,6 +45,9 @@ import java.util.List;
public class FSUtils {
private static final Logger LOG = LogManager.getLogger(FSUtils.class);
+ // Log files are of this pattern - b5068208-e1a4-11e6-bf01-fe55135034f3.avro.delta.1
+ private static final Pattern LOG_FILE_PATTERN = Pattern.compile("(.*)\\.(.*)\\.(.*)\\.([0-9]*)");
+ private static final int MAX_ATTEMPTS_RECOVER_LEASE = 10;
public static FileSystem getFs() {
Configuration conf = new Configuration();
@@ -119,4 +128,163 @@ public class FSUtils {
public static String getInstantTime(String name) {
return name.replace(getFileExtension(name), "");
}
+
+
+ /**
+ * Get the file extension from the log file
+ * @param logPath
+ * @return
+ */
+ public static String getFileExtensionFromLog(Path logPath) {
+ Matcher matcher = LOG_FILE_PATTERN.matcher(logPath.getName());
+ if(!matcher.find()) {
+ throw new InvalidHoodiePathException(logPath, "LogFile");
+ }
+ return matcher.group(2) + "." + matcher.group(3);
+ }
+
+ /**
+ * Get the first part of the file name in the log file. That will be the fileId.
+ * Log file do not have commitTime in the file name.
+ *
+ * @param path
+ * @return
+ */
+ public static String getFileIdFromLogPath(Path path) {
+ Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
+ if(!matcher.find()) {
+ throw new InvalidHoodiePathException(path, "LogFile");
+ }
+ return matcher.group(1);
+ }
+
+ /**
+ * Get the last part of the file name in the log file and convert to int.
+ *
+ * @param logPath
+ * @return
+ */
+ public static int getFileVersionFromLog(Path logPath) {
+ Matcher matcher = LOG_FILE_PATTERN.matcher(logPath.getName());
+ if(!matcher.find()) {
+ throw new InvalidHoodiePathException(logPath, "LogFile");
+ }
+ return Integer.parseInt(matcher.group(4));
+ }
+
+ public static String makeLogFileName(String fileId, String logFileExtension, int version) {
+ return String.format("%s%s.%d", fileId, logFileExtension, version);
+ }
+
+ /**
+ * Get the latest log file written from the list of log files passed in
+ *
+ * @param logFiles
+ * @return
+ */
+ public static Optional getLatestLogFile(Stream logFiles) {
+ return logFiles.sorted(Comparator
+ .comparing(s -> s.getLogVersion(),
+ Comparator.reverseOrder())).findFirst();
+ }
+
+ /**
+ * Get all the log files for the passed in FileId in the partition path
+ *
+ * @param fs
+ * @param partitionPath
+ * @param fileId
+ * @param logFileExtension
+ * @return
+ */
+ public static Stream getAllLogFiles(FileSystem fs, Path partitionPath,
+ final String fileId, final String logFileExtension) throws IOException {
+ return Arrays.stream(fs.listStatus(partitionPath,
+ path -> path.getName().startsWith(fileId) && path.getName()
+ .contains(logFileExtension))).map(HoodieLogFile::new);
+ }
+
+ /**
+ * Get the latest log version for the fileId in the partition path
+ *
+ * @param fs
+ * @param partitionPath
+ * @param fileId
+ * @param logFileExtension
+ * @return
+ * @throws IOException
+ */
+ public static Optional getLatestLogVersion(FileSystem fs, Path partitionPath,
+ final String fileId, final String logFileExtension) throws IOException {
+ Optional latestLogFile =
+ getLatestLogFile(getAllLogFiles(fs, partitionPath, fileId, logFileExtension));
+ if (latestLogFile.isPresent()) {
+ return Optional.of(latestLogFile.get().getLogVersion());
+ }
+ return Optional.empty();
+ }
+
+ public static int getCurrentLogVersion(FileSystem fs, Path partitionPath,
+ final String fileId, final String logFileExtension) throws IOException {
+ Optional currentVersion =
+ getLatestLogVersion(fs, partitionPath, fileId, logFileExtension);
+ // handle potential overflow
+ return (currentVersion.isPresent()) ? currentVersion.get() : 1;
+ }
+
+ /**
+ * computes the next log version for the specified fileId in the partition path
+ *
+ * @param fs
+ * @param partitionPath
+ * @param fileId
+ * @return
+ * @throws IOException
+ */
+ public static int computeNextLogVersion(FileSystem fs, Path partitionPath, final String fileId,
+ final String logFileExtension) throws IOException {
+ Optional currentVersion =
+ getLatestLogVersion(fs, partitionPath, fileId, logFileExtension);
+ // handle potential overflow
+ return (currentVersion.isPresent()) ? currentVersion.get() + 1 : 1;
+ }
+
+ public static int getDefaultBufferSize(final FileSystem fs) {
+ return fs.getConf().getInt("io.file.buffer.size", 4096);
+ }
+
+ public static Short getDefaultReplication(FileSystem fs, Path path) {
+ return fs.getDefaultReplication(path);
+ }
+
+ public static Long getDefaultBlockSize(FileSystem fs, Path path) {
+ return fs.getDefaultBlockSize(path);
+ }
+
+ /**
+ * When a file was opened and the task died without closing the stream, another task executor cannot open because the existing lease will be active.
+ * We will try to recover the lease, from HDFS. If a data node went down, it takes about 10 minutes for the lease to be rocovered.
+ * But if the client dies, this should be instant.
+ *
+ * @param dfs
+ * @param p
+ * @return
+ * @throws IOException
+ */
+ public static boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p)
+ throws IOException, InterruptedException {
+ LOG.info("Recover lease on dfs file " + p);
+ // initiate the recovery
+ boolean recovered = false;
+ for (int nbAttempt = 0; nbAttempt < MAX_ATTEMPTS_RECOVER_LEASE; nbAttempt++) {
+ LOG.info("Attempt " + nbAttempt + " to recover lease on dfs file " + p);
+ recovered = dfs.recoverLease(p);
+ if (recovered)
+ break;
+ // Sleep for 1 second before trying again. Typically it takes about 2-3 seconds to recover under default settings
+ Thread.sleep(1000);
+ }
+ return recovered;
+
+ }
}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/exception/InvalidHoodiePathException.java b/hoodie-common/src/main/java/com/uber/hoodie/exception/InvalidHoodiePathException.java
new file mode 100644
index 000000000..2d3f342b5
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/exception/InvalidHoodiePathException.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.exception;
+
+import org.apache.hadoop.fs.Path;
+
+public class InvalidHoodiePathException extends HoodieException {
+ public InvalidHoodiePathException(Path path, String type) {
+ super("Invalid path " + path + " of type " + type);
+ }
+}
diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HdfsTestService.java b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java
similarity index 90%
rename from hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HdfsTestService.java
rename to hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java
index a96d682ae..8dd9a799e 100644
--- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HdfsTestService.java
+++ b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java
@@ -1,20 +1,20 @@
/*
- * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
- * Licensed 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
+ * Licensed 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.
+ * 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 com.uber.hoodie.hive.util;
+package com.uber.hoodie.common.minicluster;
import com.google.common.base.Preconditions;
diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/MiniClusterUtil.java b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/MiniClusterUtil.java
new file mode 100644
index 000000000..11f9a45dd
--- /dev/null
+++ b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/MiniClusterUtil.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.common.minicluster;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.zookeeper.server.ZooKeeperServer;
+
+import java.io.IOException;
+
+public class MiniClusterUtil {
+ private static MiniDFSCluster dfsCluster;
+ private static ZooKeeperServer zkServer;
+ public static Configuration configuration;
+ public static FileSystem fileSystem;
+
+ public static void setUp() throws IOException, InterruptedException {
+ if (dfsCluster == null) {
+ HdfsTestService service = new HdfsTestService();
+ dfsCluster = service.start(true);
+ configuration = service.getHadoopConf();
+ }
+ if (zkServer == null) {
+ ZookeeperTestService zkService = new ZookeeperTestService(configuration);
+ zkServer = zkService.start();
+ }
+ fileSystem = FileSystem.get(configuration);
+ }
+
+ public static void shutdown() {
+ if (dfsCluster != null) {
+ dfsCluster.shutdown();
+ }
+ if (zkServer != null) {
+ zkServer.shutdown();
+ }
+ }
+}
diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/ZookeeperTestService.java b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/ZookeeperTestService.java
similarity index 92%
rename from hoodie-hive/src/test/java/com/uber/hoodie/hive/util/ZookeeperTestService.java
rename to hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/ZookeeperTestService.java
index 94a2cb425..4ed32f114 100644
--- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/ZookeeperTestService.java
+++ b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/ZookeeperTestService.java
@@ -1,20 +1,20 @@
/*
- * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
- * Licensed 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
+ * Licensed 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.
+ * 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 com.uber.hoodie.hive.util;
+package com.uber.hoodie.common.minicluster;
import com.google.common.base.Preconditions;
import com.google.common.io.Files;
diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/avro/AvroLogAppenderTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/avro/AvroLogAppenderTest.java
new file mode 100644
index 000000000..3764a2875
--- /dev/null
+++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/avro/AvroLogAppenderTest.java
@@ -0,0 +1,315 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.common.table.log.avro;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.uber.hoodie.common.minicluster.MiniClusterUtil;
+import com.uber.hoodie.common.table.log.HoodieLogAppendConfig;
+import com.uber.hoodie.common.table.log.HoodieLogFile;
+import com.uber.hoodie.common.util.FSUtils;
+import com.uber.hoodie.common.util.SchemaTestUtil;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.commons.collections.IteratorUtils;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.stream.Collectors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class AvroLogAppenderTest {
+ private FileSystem fs;
+ private Path partitionPath;
+
+ @BeforeClass
+ public static void setUpClass() throws IOException, InterruptedException {
+ // Append is not supported in LocalFileSystem. HDFS needs to be setup.
+ MiniClusterUtil.setUp();
+ }
+
+ @AfterClass
+ public static void tearDownClass() {
+ MiniClusterUtil.shutdown();
+ }
+
+ @Before
+ public void setUp() throws IOException, InterruptedException {
+ this.fs = MiniClusterUtil.fileSystem;
+ TemporaryFolder folder = new TemporaryFolder();
+ folder.create();
+ assertTrue(fs.mkdirs(new Path(folder.getRoot().getPath())));
+ this.partitionPath = new Path(folder.getRoot().getPath());
+ }
+
+ @After
+ public void tearDown() throws IOException {
+ fs.delete(partitionPath, true);
+ }
+
+ @Test
+ public void testBasicAppend() throws IOException, URISyntaxException, InterruptedException {
+ HoodieLogAppendConfig logConfig =
+ HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
+ .withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
+ RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
+ logAppender.append(SchemaTestUtil.generateTestRecords(0, 100));
+ long size1 = logAppender.getCurrentSize();
+ assertTrue("", size1 > 0);
+ assertEquals("", size1, fs.getFileStatus(logConfig.getLogFile().getPath()).getLen());
+ logAppender.close();
+
+ // Close and Open again and append 100 more records
+ logAppender = new RollingAvroLogAppender(logConfig);
+ logAppender.append(SchemaTestUtil.generateTestRecords(100, 100));
+ long size2 = logAppender.getCurrentSize();
+ assertTrue("", size2 > size1);
+ assertEquals("", size2, fs.getFileStatus(logConfig.getLogFile().getPath()).getLen());
+ logAppender.close();
+
+ // Close and Open again and append 100 more records
+ logAppender = new RollingAvroLogAppender(logConfig);
+ logAppender.append(SchemaTestUtil.generateTestRecords(200, 100));
+ long size3 = logAppender.getCurrentSize();
+ assertTrue("", size3 > size2);
+ assertEquals("", size3, fs.getFileStatus(logConfig.getLogFile().getPath()).getLen());
+ logAppender.close();
+ // Cannot get the current size after closing the log
+ try {
+ logAppender.getCurrentSize();
+ fail("getCurrentSize should fail after the logAppender is closed");
+ } catch (IllegalStateException e) {
+ // pass
+ }
+ }
+
+ @Test
+ public void testLeaseRecovery() throws IOException, URISyntaxException, InterruptedException {
+ HoodieLogAppendConfig logConfig =
+ HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
+ .withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
+ RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
+ logAppender.append(SchemaTestUtil.generateTestRecords(0, 100));
+ // do not close this log appender
+ // logAppender.close();
+
+ // Try opening again and append 100 more records
+ logAppender = new RollingAvroLogAppender(logConfig);
+ logAppender.append(SchemaTestUtil.generateTestRecords(100, 100));
+ assertEquals("", logAppender.getCurrentSize(),
+ fs.getFileStatus(logConfig.getLogFile().getPath()).getLen());
+ logAppender.close();
+ }
+
+ @Test
+ public void testAppendOnCorruptedBlock()
+ throws IOException, URISyntaxException, InterruptedException {
+ HoodieLogAppendConfig logConfig =
+ HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
+ .withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
+ RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
+ logAppender.append(SchemaTestUtil.generateTestRecords(0, 100));
+ logAppender.close();
+
+ // Append some arbit byte[] to thee end of the log (mimics a partially written commit)
+ assertTrue(fs.exists(logConfig.getLogFile().getPath()));
+ fs = FileSystem.get(fs.getConf());
+ FSDataOutputStream outputStream =
+ fs.append(logConfig.getLogFile().getPath(), logConfig.getBufferSize());
+ outputStream.write("something-random".getBytes());
+ outputStream.flush();
+ outputStream.close();
+
+ logAppender = new RollingAvroLogAppender(logConfig);
+ logAppender.append(SchemaTestUtil.generateTestRecords(100, 100));
+ logAppender.close();
+ }
+
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testBasicWriteAndRead()
+ throws IOException, URISyntaxException, InterruptedException {
+ HoodieLogAppendConfig logConfig =
+ HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
+ .withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
+ RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
+ long size1 = logAppender.getCurrentSize();
+
+ List inputRecords = SchemaTestUtil.generateTestRecords(0, 100);
+ logAppender.append(inputRecords);
+ logAppender.close();
+
+ AvroLogReader logReader =
+ new AvroLogReader(logConfig.getLogFile(), fs, logConfig.getSchema());
+ List result = IteratorUtils.toList(logReader.readBlock(size1));
+ assertEquals("Random access should return 100 records", 100, result.size());
+ assertEquals("both lists should be the same. (ordering guaranteed)", inputRecords, result);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testBasicAppendAndRead()
+ throws IOException, URISyntaxException, InterruptedException {
+ HoodieLogAppendConfig logConfig =
+ HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
+ .withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
+ RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
+ logAppender.append(SchemaTestUtil.generateTestRecords(0, 100));
+ long size1 = logAppender.getCurrentSize();
+ logAppender.close();
+
+ // Close and Open again and append 100 more records
+ logAppender = new RollingAvroLogAppender(logConfig);
+ List secondBatchInput = SchemaTestUtil.generateTestRecords(100, 100);
+ logAppender.append(secondBatchInput);
+ long size2 = logAppender.getCurrentSize();
+ logAppender.close();
+
+ // Close and Open again and append 100 more records
+ logAppender = new RollingAvroLogAppender(logConfig);
+ List lastBatchInput = SchemaTestUtil.generateTestRecords(200, 100);
+ logAppender.append(lastBatchInput);
+ long size3 = logAppender.getCurrentSize();
+ logAppender.close();
+
+ AvroLogReader logReader =
+ new AvroLogReader(logConfig.getLogFile(), fs, logConfig.getSchema());
+
+ // Try to grab the middle block here
+ List secondBatch = IteratorUtils.toList(logReader.readBlock(size1));
+ assertEquals("Stream collect should return 100 records", 100, secondBatch.size());
+ assertEquals("Collected list should match the input list (ordering guaranteed)",
+ secondBatchInput, secondBatch);
+
+ // Try to grab the middle block here
+ List lastBatch = IteratorUtils.toList(logReader.readBlock(size2));
+ assertEquals("Stream collect should return 100 records", 100, secondBatch.size());
+ assertEquals("Collected list should match the input list (ordering guaranteed)",
+ lastBatchInput, lastBatch);
+
+ List imaginaryBatch = IteratorUtils.toList(logReader.readBlock(size3));
+ assertEquals("Stream collect should return 0 records", 0, imaginaryBatch.size());
+ }
+
+ @Test
+ public void testAppendAndReadOnCorruptedLog()
+ throws IOException, URISyntaxException, InterruptedException {
+ HoodieLogAppendConfig logConfig =
+ HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
+ .withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
+ RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
+ long size1 = logAppender.getCurrentSize();
+ logAppender.append(SchemaTestUtil.generateTestRecords(0, 100));
+ logAppender.close();
+
+ // Append some arbit byte[] to thee end of the log (mimics a partially written commit)
+ assertTrue(fs.exists(logConfig.getLogFile().getPath()));
+ fs = FileSystem.get(fs.getConf());
+ FSDataOutputStream outputStream =
+ fs.append(logConfig.getLogFile().getPath(), logConfig.getBufferSize());
+ outputStream.write("something-random".getBytes());
+ outputStream.flush();
+ outputStream.close();
+
+ logAppender = new RollingAvroLogAppender(logConfig);
+ long size2 = logAppender.getCurrentSize();
+ logAppender.append(SchemaTestUtil.generateTestRecords(100, 100));
+ logAppender.close();
+
+ AvroLogReader logReader =
+ new AvroLogReader(logConfig.getLogFile(), fs, logConfig.getSchema());
+
+ // Try to grab the middle block here
+ List secondBatch = IteratorUtils.toList(logReader.readBlock(size1));
+ assertEquals("Stream collect should return 100 records", 100, secondBatch.size());
+
+ // Try to grab the last block here
+ List lastBatch = IteratorUtils.toList(logReader.readBlock(size2));
+ assertEquals("Stream collect should return 100 records", 100, lastBatch.size());
+ }
+
+ @Test
+ public void testCompositeAvroLogReader()
+ throws IOException, URISyntaxException, InterruptedException {
+ // Set a small threshold so that every block is a new version
+ HoodieLogAppendConfig logConfig =
+ HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
+ .withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .withSchema(SchemaTestUtil.getSimpleSchema()).withSizeThreshold(500).withFs(fs)
+ .build();
+
+ RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
+ long size1 = logAppender.getCurrentSize();
+ List input1 = SchemaTestUtil.generateTestRecords(0, 100);
+ logAppender.append(input1);
+ logAppender.close();
+
+ // Need to rebuild config to set the latest version as path
+ logConfig = HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
+ .withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .withSchema(SchemaTestUtil.getSimpleSchema()).withSizeThreshold(500).withFs(fs).build();
+ logAppender = new RollingAvroLogAppender(logConfig);
+ long size2 = logAppender.getCurrentSize();
+ List input2 = SchemaTestUtil.generateTestRecords(100, 100);
+ logAppender.append(input2);
+ logAppender.close();
+
+ logConfig = HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
+ .withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
+ .withSchema(SchemaTestUtil.getSimpleSchema()).withSizeThreshold(500).withFs(fs).build();
+ List allLogFiles = FSUtils
+ .getAllLogFiles(fs, partitionPath, logConfig.getLogFile().getFileId(),
+ HoodieLogFile.DELTA_EXTENSION).collect(Collectors.toList());
+ assertEquals("", 2, allLogFiles.size());
+
+ SortedMap> offsets = Maps.newTreeMap();
+ offsets.put(1, Lists.newArrayList(size1));
+ offsets.put(2, Lists.newArrayList(size2));
+ CompositeAvroLogReader reader =
+ new CompositeAvroLogReader(partitionPath, logConfig.getLogFile().getFileId(), fs,
+ logConfig.getSchema(), HoodieLogFile.DELTA_EXTENSION);
+ Iterator results = reader.readBlocks(offsets);
+ List totalBatch = IteratorUtils.toList(results);
+ assertEquals("Stream collect should return all 200 records", 200, totalBatch.size());
+ input1.addAll(input2);
+ assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", input1,
+ totalBatch);
+ }
+}
diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java
new file mode 100644
index 000000000..fbec264fd
--- /dev/null
+++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed 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 com.uber.hoodie.common.util;
+
+import com.uber.hoodie.exception.HoodieIOException;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.io.DecoderFactory;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public class SchemaTestUtil {
+ public static Schema getSimpleSchema() throws IOException {
+ return new Schema.Parser()
+ .parse(SchemaTestUtil.class.getResourceAsStream("/simple-test.avro"));
+ }
+
+ public static List generateTestRecords(int from, int limit)
+ throws IOException, URISyntaxException {
+ return toRecords(getSimpleSchema(), getSimpleSchema(), from, limit);
+ }
+
+ private static List toRecords(Schema writerSchema, Schema readerSchema, int from,
+ int limit) throws IOException, URISyntaxException {
+ GenericDatumReader reader =
+ new GenericDatumReader<>(writerSchema, readerSchema);
+ try (Stream stream = Files
+ .lines(Paths.get(SchemaTestUtil.class.getResource("/sample.data").toURI()))) {
+ return stream.skip(from).limit(limit).map(s -> {
+ try {
+ return reader.read(null, DecoderFactory.get().jsonDecoder(readerSchema, s));
+ } catch (IOException e) {
+ throw new HoodieIOException("Could not read data from simple_data.json", e);
+ }
+ }).collect(Collectors.toList());
+ } catch (IOException e) {
+ throw new HoodieIOException("Could not read data from simple_data.json", e);
+ }
+ }
+
+}
diff --git a/hoodie-common/src/test/resources/sample.data b/hoodie-common/src/test/resources/sample.data
new file mode 100644
index 000000000..29c816899
--- /dev/null
+++ b/hoodie-common/src/test/resources/sample.data
@@ -0,0 +1,1000 @@
+{"name":"tellus","favorite_number":47,"favorite_color":"Turquoise"}
+{"name":"dapibus","favorite_number":85,"favorite_color":"Crimson"}
+{"name":"sapien","favorite_number":8,"favorite_color":"Mauv"}
+{"name":"integer","favorite_number":100,"favorite_color":"Indigo"}
+{"name":"velit","favorite_number":40,"favorite_color":"Green"}
+{"name":"justo","favorite_number":99,"favorite_color":"Blue"}
+{"name":"nulla","favorite_number":73,"favorite_color":"Mauv"}
+{"name":"porttitor","favorite_number":98,"favorite_color":"Red"}
+{"name":"vel","favorite_number":44,"favorite_color":"Yellow"}
+{"name":"sociis","favorite_number":50,"favorite_color":"Green"}
+{"name":"venenatis","favorite_number":32,"favorite_color":"Pink"}
+{"name":"ac","favorite_number":8,"favorite_color":"Puce"}
+{"name":"quam","favorite_number":55,"favorite_color":"Maroon"}
+{"name":"eu","favorite_number":69,"favorite_color":"Green"}
+{"name":"vestibulum","favorite_number":49,"favorite_color":"Fuscia"}
+{"name":"a","favorite_number":22,"favorite_color":"Teal"}
+{"name":"proin","favorite_number":78,"favorite_color":"Goldenrod"}
+{"name":"interdum","favorite_number":89,"favorite_color":"Fuscia"}
+{"name":"turpis","favorite_number":29,"favorite_color":"Teal"}
+{"name":"nullam","favorite_number":36,"favorite_color":"Maroon"}
+{"name":"nullam","favorite_number":89,"favorite_color":"Violet"}
+{"name":"sed","favorite_number":36,"favorite_color":"Indigo"}
+{"name":"id","favorite_number":61,"favorite_color":"Violet"}
+{"name":"sapien","favorite_number":58,"favorite_color":"Teal"}
+{"name":"dapibus","favorite_number":37,"favorite_color":"Khaki"}
+{"name":"id","favorite_number":5,"favorite_color":"Blue"}
+{"name":"condimentum","favorite_number":61,"favorite_color":"Teal"}
+{"name":"orci","favorite_number":78,"favorite_color":"Orange"}
+{"name":"in","favorite_number":14,"favorite_color":"Violet"}
+{"name":"proin","favorite_number":94,"favorite_color":"Purple"}
+{"name":"semper","favorite_number":53,"favorite_color":"Indigo"}
+{"name":"tortor","favorite_number":50,"favorite_color":"Red"}
+{"name":"quam","favorite_number":78,"favorite_color":"Red"}
+{"name":"semper","favorite_number":2,"favorite_color":"Violet"}
+{"name":"in","favorite_number":24,"favorite_color":"Khaki"}
+{"name":"sapien","favorite_number":10,"favorite_color":"Crimson"}
+{"name":"posuere","favorite_number":88,"favorite_color":"Maroon"}
+{"name":"eleifend","favorite_number":72,"favorite_color":"Yellow"}
+{"name":"urna","favorite_number":56,"favorite_color":"Goldenrod"}
+{"name":"duis","favorite_number":49,"favorite_color":"Crimson"}
+{"name":"pellentesque","favorite_number":69,"favorite_color":"Turquoise"}
+{"name":"facilisi","favorite_number":33,"favorite_color":"Maroon"}
+{"name":"aenean","favorite_number":1,"favorite_color":"Fuscia"}
+{"name":"vestibulum","favorite_number":51,"favorite_color":"Green"}
+{"name":"sagittis","favorite_number":86,"favorite_color":"Teal"}
+{"name":"in","favorite_number":86,"favorite_color":"Indigo"}
+{"name":"platea","favorite_number":68,"favorite_color":"Indigo"}
+{"name":"phasellus","favorite_number":65,"favorite_color":"Yellow"}
+{"name":"ultricies","favorite_number":42,"favorite_color":"Yellow"}
+{"name":"natoque","favorite_number":69,"favorite_color":"Red"}
+{"name":"ligula","favorite_number":5,"favorite_color":"Pink"}
+{"name":"duis","favorite_number":11,"favorite_color":"Goldenrod"}
+{"name":"dapibus","favorite_number":53,"favorite_color":"Teal"}
+{"name":"morbi","favorite_number":22,"favorite_color":"Red"}
+{"name":"id","favorite_number":16,"favorite_color":"Teal"}
+{"name":"consequat","favorite_number":67,"favorite_color":"Turquoise"}
+{"name":"semper","favorite_number":40,"favorite_color":"Yellow"}
+{"name":"sit","favorite_number":6,"favorite_color":"Crimson"}
+{"name":"sollicitudin","favorite_number":25,"favorite_color":"Yellow"}
+{"name":"aliquam","favorite_number":39,"favorite_color":"Maroon"}
+{"name":"ultricies","favorite_number":43,"favorite_color":"Puce"}
+{"name":"lorem","favorite_number":33,"favorite_color":"Pink"}
+{"name":"lectus","favorite_number":3,"favorite_color":"Purple"}
+{"name":"velit","favorite_number":94,"favorite_color":"Pink"}
+{"name":"ac","favorite_number":58,"favorite_color":"Red"}
+{"name":"lorem","favorite_number":14,"favorite_color":"Teal"}
+{"name":"tempor","favorite_number":28,"favorite_color":"Goldenrod"}
+{"name":"ut","favorite_number":16,"favorite_color":"Khaki"}
+{"name":"eget","favorite_number":23,"favorite_color":"Blue"}
+{"name":"quis","favorite_number":93,"favorite_color":"Orange"}
+{"name":"montes","favorite_number":77,"favorite_color":"Green"}
+{"name":"faucibus","favorite_number":42,"favorite_color":"Purple"}
+{"name":"faucibus","favorite_number":3,"favorite_color":"Khaki"}
+{"name":"ac","favorite_number":71,"favorite_color":"Indigo"}
+{"name":"dapibus","favorite_number":81,"favorite_color":"Mauv"}
+{"name":"accumsan","favorite_number":92,"favorite_color":"Pink"}
+{"name":"suspendisse","favorite_number":20,"favorite_color":"Aquamarine"}
+{"name":"sit","favorite_number":91,"favorite_color":"Maroon"}
+{"name":"imperdiet","favorite_number":20,"favorite_color":"Red"}
+{"name":"ac","favorite_number":37,"favorite_color":"Mauv"}
+{"name":"sociis","favorite_number":97,"favorite_color":"Turquoise"}
+{"name":"lacinia","favorite_number":35,"favorite_color":"Maroon"}
+{"name":"curabitur","favorite_number":51,"favorite_color":"Orange"}
+{"name":"sapien","favorite_number":37,"favorite_color":"Crimson"}
+{"name":"in","favorite_number":27,"favorite_color":"Crimson"}
+{"name":"accumsan","favorite_number":35,"favorite_color":"Khaki"}
+{"name":"adipiscing","favorite_number":9,"favorite_color":"Green"}
+{"name":"duis","favorite_number":85,"favorite_color":"Green"}
+{"name":"tellus","favorite_number":46,"favorite_color":"Indigo"}
+{"name":"in","favorite_number":21,"favorite_color":"Puce"}
+{"name":"lobortis","favorite_number":79,"favorite_color":"Teal"}
+{"name":"dapibus","favorite_number":88,"favorite_color":"Indigo"}
+{"name":"curabitur","favorite_number":71,"favorite_color":"Fuscia"}
+{"name":"sapien","favorite_number":97,"favorite_color":"Turquoise"}
+{"name":"semper","favorite_number":75,"favorite_color":"Turquoise"}
+{"name":"a","favorite_number":95,"favorite_color":"Aquamarine"}
+{"name":"interdum","favorite_number":81,"favorite_color":"Pink"}
+{"name":"dapibus","favorite_number":40,"favorite_color":"Crimson"}
+{"name":"vivamus","favorite_number":16,"favorite_color":"Orange"}
+{"name":"accumsan","favorite_number":39,"favorite_color":"Aquamarine"}
+{"name":"imperdiet","favorite_number":74,"favorite_color":"Fuscia"}
+{"name":"quis","favorite_number":38,"favorite_color":"Turquoise"}
+{"name":"libero","favorite_number":90,"favorite_color":"Puce"}
+{"name":"laoreet","favorite_number":69,"favorite_color":"Puce"}
+{"name":"in","favorite_number":84,"favorite_color":"Maroon"}
+{"name":"eros","favorite_number":98,"favorite_color":"Blue"}
+{"name":"montes","favorite_number":92,"favorite_color":"Goldenrod"}
+{"name":"amet","favorite_number":46,"favorite_color":"Red"}
+{"name":"vulputate","favorite_number":24,"favorite_color":"Turquoise"}
+{"name":"primis","favorite_number":6,"favorite_color":"Purple"}
+{"name":"leo","favorite_number":59,"favorite_color":"Purple"}
+{"name":"nec","favorite_number":85,"favorite_color":"Turquoise"}
+{"name":"mauris","favorite_number":91,"favorite_color":"Turquoise"}
+{"name":"ligula","favorite_number":60,"favorite_color":"Goldenrod"}
+{"name":"morbi","favorite_number":15,"favorite_color":"Turquoise"}
+{"name":"consequat","favorite_number":34,"favorite_color":"Aquamarine"}
+{"name":"nunc","favorite_number":91,"favorite_color":"Pink"}
+{"name":"at","favorite_number":85,"favorite_color":"Yellow"}
+{"name":"nunc","favorite_number":57,"favorite_color":"Orange"}
+{"name":"sed","favorite_number":35,"favorite_color":"Crimson"}
+{"name":"fusce","favorite_number":87,"favorite_color":"Blue"}
+{"name":"nibh","favorite_number":32,"favorite_color":"Goldenrod"}
+{"name":"mi","favorite_number":66,"favorite_color":"Mauv"}
+{"name":"duis","favorite_number":5,"favorite_color":"Turquoise"}
+{"name":"nisl","favorite_number":80,"favorite_color":"Violet"}
+{"name":"orci","favorite_number":72,"favorite_color":"Indigo"}
+{"name":"semper","favorite_number":39,"favorite_color":"Indigo"}
+{"name":"in","favorite_number":94,"favorite_color":"Orange"}
+{"name":"ultrices","favorite_number":3,"favorite_color":"Mauv"}
+{"name":"integer","favorite_number":74,"favorite_color":"Puce"}
+{"name":"ante","favorite_number":74,"favorite_color":"Aquamarine"}
+{"name":"in","favorite_number":54,"favorite_color":"Goldenrod"}
+{"name":"pretium","favorite_number":95,"favorite_color":"Crimson"}
+{"name":"in","favorite_number":74,"favorite_color":"Maroon"}
+{"name":"a","favorite_number":48,"favorite_color":"Violet"}
+{"name":"blandit","favorite_number":26,"favorite_color":"Yellow"}
+{"name":"duis","favorite_number":49,"favorite_color":"Aquamarine"}
+{"name":"nisi","favorite_number":54,"favorite_color":"Khaki"}
+{"name":"natoque","favorite_number":29,"favorite_color":"Maroon"}
+{"name":"erat","favorite_number":17,"favorite_color":"Mauv"}
+{"name":"in","favorite_number":93,"favorite_color":"Turquoise"}
+{"name":"sed","favorite_number":48,"favorite_color":"Puce"}
+{"name":"vestibulum","favorite_number":27,"favorite_color":"Maroon"}
+{"name":"vel","favorite_number":95,"favorite_color":"Aquamarine"}
+{"name":"tincidunt","favorite_number":95,"favorite_color":"Mauv"}
+{"name":"sit","favorite_number":4,"favorite_color":"Teal"}
+{"name":"justo","favorite_number":93,"favorite_color":"Green"}
+{"name":"justo","favorite_number":17,"favorite_color":"Maroon"}
+{"name":"eu","favorite_number":9,"favorite_color":"Yellow"}
+{"name":"metus","favorite_number":37,"favorite_color":"Purple"}
+{"name":"nulla","favorite_number":55,"favorite_color":"Khaki"}
+{"name":"adipiscing","favorite_number":32,"favorite_color":"Crimson"}
+{"name":"non","favorite_number":72,"favorite_color":"Teal"}
+{"name":"quam","favorite_number":92,"favorite_color":"Pink"}
+{"name":"maecenas","favorite_number":73,"favorite_color":"Puce"}
+{"name":"curae","favorite_number":7,"favorite_color":"Goldenrod"}
+{"name":"faucibus","favorite_number":81,"favorite_color":"Blue"}
+{"name":"lectus","favorite_number":27,"favorite_color":"Teal"}
+{"name":"rutrum","favorite_number":24,"favorite_color":"Pink"}
+{"name":"convallis","favorite_number":12,"favorite_color":"Yellow"}
+{"name":"primis","favorite_number":31,"favorite_color":"Indigo"}
+{"name":"convallis","favorite_number":9,"favorite_color":"Khaki"}
+{"name":"cras","favorite_number":8,"favorite_color":"Pink"}
+{"name":"libero","favorite_number":44,"favorite_color":"Puce"}
+{"name":"magna","favorite_number":57,"favorite_color":"Violet"}
+{"name":"et","favorite_number":6,"favorite_color":"Puce"}
+{"name":"sit","favorite_number":97,"favorite_color":"Mauv"}
+{"name":"a","favorite_number":58,"favorite_color":"Red"}
+{"name":"platea","favorite_number":50,"favorite_color":"Crimson"}
+{"name":"adipiscing","favorite_number":85,"favorite_color":"Khaki"}
+{"name":"vel","favorite_number":94,"favorite_color":"Goldenrod"}
+{"name":"leo","favorite_number":78,"favorite_color":"Turquoise"}
+{"name":"suscipit","favorite_number":6,"favorite_color":"Violet"}
+{"name":"integer","favorite_number":37,"favorite_color":"Blue"}
+{"name":"justo","favorite_number":87,"favorite_color":"Yellow"}
+{"name":"gravida","favorite_number":24,"favorite_color":"Fuscia"}
+{"name":"sit","favorite_number":64,"favorite_color":"Purple"}
+{"name":"mauris","favorite_number":14,"favorite_color":"Maroon"}
+{"name":"cubilia","favorite_number":3,"favorite_color":"Indigo"}
+{"name":"et","favorite_number":10,"favorite_color":"Green"}
+{"name":"luctus","favorite_number":91,"favorite_color":"Purple"}
+{"name":"pulvinar","favorite_number":84,"favorite_color":"Green"}
+{"name":"erat","favorite_number":13,"favorite_color":"Aquamarine"}
+{"name":"molestie","favorite_number":12,"favorite_color":"Mauv"}
+{"name":"tincidunt","favorite_number":19,"favorite_color":"Fuscia"}
+{"name":"sed","favorite_number":13,"favorite_color":"Teal"}
+{"name":"magna","favorite_number":53,"favorite_color":"Green"}
+{"name":"mattis","favorite_number":44,"favorite_color":"Fuscia"}
+{"name":"vestibulum","favorite_number":23,"favorite_color":"Violet"}
+{"name":"pede","favorite_number":31,"favorite_color":"Violet"}
+{"name":"lorem","favorite_number":92,"favorite_color":"Maroon"}
+{"name":"nibh","favorite_number":76,"favorite_color":"Green"}
+{"name":"id","favorite_number":99,"favorite_color":"Turquoise"}
+{"name":"dapibus","favorite_number":25,"favorite_color":"Pink"}
+{"name":"nulla","favorite_number":25,"favorite_color":"Blue"}
+{"name":"in","favorite_number":37,"favorite_color":"Crimson"}
+{"name":"lacinia","favorite_number":71,"favorite_color":"Green"}
+{"name":"nibh","favorite_number":96,"favorite_color":"Violet"}
+{"name":"ante","favorite_number":38,"favorite_color":"Aquamarine"}
+{"name":"quis","favorite_number":42,"favorite_color":"Turquoise"}
+{"name":"magna","favorite_number":66,"favorite_color":"Yellow"}
+{"name":"curae","favorite_number":88,"favorite_color":"Crimson"}
+{"name":"ut","favorite_number":58,"favorite_color":"Aquamarine"}
+{"name":"diam","favorite_number":22,"favorite_color":"Blue"}
+{"name":"fermentum","favorite_number":19,"favorite_color":"Indigo"}
+{"name":"duis","favorite_number":51,"favorite_color":"Goldenrod"}
+{"name":"ante","favorite_number":14,"favorite_color":"Indigo"}
+{"name":"integer","favorite_number":12,"favorite_color":"Aquamarine"}
+{"name":"nonummy","favorite_number":91,"favorite_color":"Indigo"}
+{"name":"magnis","favorite_number":65,"favorite_color":"Fuscia"}
+{"name":"commodo","favorite_number":25,"favorite_color":"Khaki"}
+{"name":"sapien","favorite_number":88,"favorite_color":"Pink"}
+{"name":"quam","favorite_number":31,"favorite_color":"Red"}
+{"name":"justo","favorite_number":55,"favorite_color":"Aquamarine"}
+{"name":"dolor","favorite_number":45,"favorite_color":"Pink"}
+{"name":"at","favorite_number":54,"favorite_color":"Purple"}
+{"name":"et","favorite_number":37,"favorite_color":"Puce"}
+{"name":"nulla","favorite_number":60,"favorite_color":"Mauv"}
+{"name":"mauris","favorite_number":50,"favorite_color":"Mauv"}
+{"name":"etiam","favorite_number":58,"favorite_color":"Pink"}
+{"name":"nulla","favorite_number":73,"favorite_color":"Green"}
+{"name":"natoque","favorite_number":63,"favorite_color":"Maroon"}
+{"name":"mauris","favorite_number":88,"favorite_color":"Mauv"}
+{"name":"rhoncus","favorite_number":78,"favorite_color":"Teal"}
+{"name":"bibendum","favorite_number":60,"favorite_color":"Purple"}
+{"name":"accumsan","favorite_number":2,"favorite_color":"Aquamarine"}
+{"name":"ut","favorite_number":49,"favorite_color":"Indigo"}
+{"name":"eleifend","favorite_number":17,"favorite_color":"Aquamarine"}
+{"name":"morbi","favorite_number":25,"favorite_color":"Crimson"}
+{"name":"non","favorite_number":33,"favorite_color":"Aquamarine"}
+{"name":"pretium","favorite_number":50,"favorite_color":"Crimson"}
+{"name":"lacinia","favorite_number":17,"favorite_color":"Indigo"}
+{"name":"mus","favorite_number":100,"favorite_color":"Red"}
+{"name":"leo","favorite_number":71,"favorite_color":"Purple"}
+{"name":"justo","favorite_number":2,"favorite_color":"Indigo"}
+{"name":"fermentum","favorite_number":58,"favorite_color":"Fuscia"}
+{"name":"eu","favorite_number":4,"favorite_color":"Puce"}
+{"name":"cubilia","favorite_number":91,"favorite_color":"Green"}
+{"name":"convallis","favorite_number":61,"favorite_color":"Green"}
+{"name":"posuere","favorite_number":83,"favorite_color":"Green"}
+{"name":"diam","favorite_number":75,"favorite_color":"Khaki"}
+{"name":"vestibulum","favorite_number":9,"favorite_color":"Pink"}
+{"name":"sagittis","favorite_number":14,"favorite_color":"Green"}
+{"name":"nullam","favorite_number":93,"favorite_color":"Violet"}
+{"name":"a","favorite_number":23,"favorite_color":"Puce"}
+{"name":"proin","favorite_number":59,"favorite_color":"Puce"}
+{"name":"ipsum","favorite_number":89,"favorite_color":"Maroon"}
+{"name":"sapien","favorite_number":97,"favorite_color":"Pink"}
+{"name":"in","favorite_number":20,"favorite_color":"Teal"}
+{"name":"lectus","favorite_number":85,"favorite_color":"Crimson"}
+{"name":"tincidunt","favorite_number":62,"favorite_color":"Aquamarine"}
+{"name":"at","favorite_number":86,"favorite_color":"Red"}
+{"name":"turpis","favorite_number":17,"favorite_color":"Khaki"}
+{"name":"nisi","favorite_number":100,"favorite_color":"Yellow"}
+{"name":"sed","favorite_number":17,"favorite_color":"Khaki"}
+{"name":"sem","favorite_number":60,"favorite_color":"Goldenrod"}
+{"name":"sapien","favorite_number":84,"favorite_color":"Indigo"}
+{"name":"tempus","favorite_number":95,"favorite_color":"Teal"}
+{"name":"erat","favorite_number":18,"favorite_color":"Purple"}
+{"name":"ipsum","favorite_number":72,"favorite_color":"Goldenrod"}
+{"name":"aenean","favorite_number":15,"favorite_color":"Crimson"}
+{"name":"est","favorite_number":76,"favorite_color":"Yellow"}
+{"name":"vel","favorite_number":33,"favorite_color":"Indigo"}
+{"name":"vestibulum","favorite_number":78,"favorite_color":"Purple"}
+{"name":"vel","favorite_number":100,"favorite_color":"Green"}
+{"name":"eget","favorite_number":89,"favorite_color":"Crimson"}
+{"name":"et","favorite_number":50,"favorite_color":"Maroon"}
+{"name":"lectus","favorite_number":100,"favorite_color":"Crimson"}
+{"name":"sed","favorite_number":100,"favorite_color":"Teal"}
+{"name":"dolor","favorite_number":84,"favorite_color":"Indigo"}
+{"name":"pede","favorite_number":23,"favorite_color":"Purple"}
+{"name":"ipsum","favorite_number":15,"favorite_color":"Teal"}
+{"name":"cubilia","favorite_number":51,"favorite_color":"Green"}
+{"name":"mi","favorite_number":50,"favorite_color":"Yellow"}
+{"name":"porttitor","favorite_number":47,"favorite_color":"Goldenrod"}
+{"name":"hac","favorite_number":46,"favorite_color":"Goldenrod"}
+{"name":"lectus","favorite_number":24,"favorite_color":"Khaki"}
+{"name":"orci","favorite_number":2,"favorite_color":"Violet"}
+{"name":"mauris","favorite_number":62,"favorite_color":"Blue"}
+{"name":"hac","favorite_number":32,"favorite_color":"Teal"}
+{"name":"accumsan","favorite_number":69,"favorite_color":"Puce"}
+{"name":"maecenas","favorite_number":59,"favorite_color":"Turquoise"}
+{"name":"arcu","favorite_number":3,"favorite_color":"Mauv"}
+{"name":"integer","favorite_number":98,"favorite_color":"Khaki"}
+{"name":"justo","favorite_number":73,"favorite_color":"Violet"}
+{"name":"non","favorite_number":59,"favorite_color":"Teal"}
+{"name":"justo","favorite_number":6,"favorite_color":"Maroon"}
+{"name":"turpis","favorite_number":89,"favorite_color":"Teal"}
+{"name":"orci","favorite_number":63,"favorite_color":"Red"}
+{"name":"eros","favorite_number":27,"favorite_color":"Turquoise"}
+{"name":"ligula","favorite_number":17,"favorite_color":"Goldenrod"}
+{"name":"at","favorite_number":16,"favorite_color":"Crimson"}
+{"name":"amet","favorite_number":94,"favorite_color":"Teal"}
+{"name":"ullamcorper","favorite_number":68,"favorite_color":"Violet"}
+{"name":"enim","favorite_number":9,"favorite_color":"Khaki"}
+{"name":"lacus","favorite_number":67,"favorite_color":"Goldenrod"}
+{"name":"mattis","favorite_number":56,"favorite_color":"Mauv"}
+{"name":"neque","favorite_number":97,"favorite_color":"Mauv"}
+{"name":"rhoncus","favorite_number":67,"favorite_color":"Maroon"}
+{"name":"maecenas","favorite_number":19,"favorite_color":"Blue"}
+{"name":"eget","favorite_number":11,"favorite_color":"Violet"}
+{"name":"ante","favorite_number":98,"favorite_color":"Mauv"}
+{"name":"justo","favorite_number":69,"favorite_color":"Turquoise"}
+{"name":"morbi","favorite_number":21,"favorite_color":"Crimson"}
+{"name":"lorem","favorite_number":83,"favorite_color":"Aquamarine"}
+{"name":"nulla","favorite_number":75,"favorite_color":"Pink"}
+{"name":"augue","favorite_number":88,"favorite_color":"Violet"}
+{"name":"id","favorite_number":14,"favorite_color":"Turquoise"}
+{"name":"diam","favorite_number":56,"favorite_color":"Turquoise"}
+{"name":"ut","favorite_number":20,"favorite_color":"Teal"}
+{"name":"magna","favorite_number":66,"favorite_color":"Turquoise"}
+{"name":"volutpat","favorite_number":22,"favorite_color":"Teal"}
+{"name":"tortor","favorite_number":76,"favorite_color":"Fuscia"}
+{"name":"mauris","favorite_number":61,"favorite_color":"Mauv"}
+{"name":"nec","favorite_number":22,"favorite_color":"Maroon"}
+{"name":"quisque","favorite_number":13,"favorite_color":"Pink"}
+{"name":"id","favorite_number":21,"favorite_color":"Khaki"}
+{"name":"id","favorite_number":25,"favorite_color":"Khaki"}
+{"name":"ipsum","favorite_number":68,"favorite_color":"Aquamarine"}
+{"name":"mattis","favorite_number":11,"favorite_color":"Aquamarine"}
+{"name":"nulla","favorite_number":27,"favorite_color":"Purple"}
+{"name":"consectetuer","favorite_number":15,"favorite_color":"Goldenrod"}
+{"name":"tellus","favorite_number":90,"favorite_color":"Green"}
+{"name":"curae","favorite_number":13,"favorite_color":"Fuscia"}
+{"name":"tempus","favorite_number":58,"favorite_color":"Aquamarine"}
+{"name":"nibh","favorite_number":99,"favorite_color":"Blue"}
+{"name":"pellentesque","favorite_number":49,"favorite_color":"Goldenrod"}
+{"name":"augue","favorite_number":16,"favorite_color":"Mauv"}
+{"name":"tellus","favorite_number":46,"favorite_color":"Orange"}
+{"name":"in","favorite_number":32,"favorite_color":"Mauv"}
+{"name":"adipiscing","favorite_number":88,"favorite_color":"Blue"}
+{"name":"donec","favorite_number":91,"favorite_color":"Purple"}
+{"name":"mauris","favorite_number":21,"favorite_color":"Violet"}
+{"name":"aenean","favorite_number":56,"favorite_color":"Violet"}
+{"name":"porta","favorite_number":4,"favorite_color":"Yellow"}
+{"name":"enim","favorite_number":88,"favorite_color":"Mauv"}
+{"name":"convallis","favorite_number":4,"favorite_color":"Turquoise"}
+{"name":"proin","favorite_number":11,"favorite_color":"Aquamarine"}
+{"name":"at","favorite_number":76,"favorite_color":"Indigo"}
+{"name":"felis","favorite_number":16,"favorite_color":"Maroon"}
+{"name":"ac","favorite_number":62,"favorite_color":"Khaki"}
+{"name":"metus","favorite_number":27,"favorite_color":"Aquamarine"}
+{"name":"sem","favorite_number":38,"favorite_color":"Indigo"}
+{"name":"lobortis","favorite_number":88,"favorite_color":"Goldenrod"}
+{"name":"sapien","favorite_number":28,"favorite_color":"Maroon"}
+{"name":"faucibus","favorite_number":45,"favorite_color":"Green"}
+{"name":"dolor","favorite_number":92,"favorite_color":"Maroon"}
+{"name":"porta","favorite_number":99,"favorite_color":"Puce"}
+{"name":"mi","favorite_number":46,"favorite_color":"Red"}
+{"name":"integer","favorite_number":45,"favorite_color":"Yellow"}
+{"name":"placerat","favorite_number":18,"favorite_color":"Green"}
+{"name":"mauris","favorite_number":28,"favorite_color":"Mauv"}
+{"name":"aliquam","favorite_number":66,"favorite_color":"Purple"}
+{"name":"sit","favorite_number":20,"favorite_color":"Violet"}
+{"name":"bibendum","favorite_number":21,"favorite_color":"Fuscia"}
+{"name":"cum","favorite_number":73,"favorite_color":"Green"}
+{"name":"tellus","favorite_number":75,"favorite_color":"Aquamarine"}
+{"name":"volutpat","favorite_number":34,"favorite_color":"Crimson"}
+{"name":"pellentesque","favorite_number":98,"favorite_color":"Khaki"}
+{"name":"lorem","favorite_number":92,"favorite_color":"Goldenrod"}
+{"name":"luctus","favorite_number":93,"favorite_color":"Crimson"}
+{"name":"parturient","favorite_number":71,"favorite_color":"Fuscia"}
+{"name":"aenean","favorite_number":22,"favorite_color":"Goldenrod"}
+{"name":"imperdiet","favorite_number":40,"favorite_color":"Crimson"}
+{"name":"nulla","favorite_number":12,"favorite_color":"Maroon"}
+{"name":"adipiscing","favorite_number":76,"favorite_color":"Maroon"}
+{"name":"ac","favorite_number":12,"favorite_color":"Goldenrod"}
+{"name":"nec","favorite_number":51,"favorite_color":"Indigo"}
+{"name":"natoque","favorite_number":76,"favorite_color":"Pink"}
+{"name":"consequat","favorite_number":10,"favorite_color":"Pink"}
+{"name":"faucibus","favorite_number":84,"favorite_color":"Turquoise"}
+{"name":"est","favorite_number":84,"favorite_color":"Teal"}
+{"name":"ac","favorite_number":96,"favorite_color":"Goldenrod"}
+{"name":"amet","favorite_number":75,"favorite_color":"Mauv"}
+{"name":"vel","favorite_number":36,"favorite_color":"Puce"}
+{"name":"cum","favorite_number":77,"favorite_color":"Violet"}
+{"name":"eget","favorite_number":90,"favorite_color":"Mauv"}
+{"name":"nullam","favorite_number":96,"favorite_color":"Teal"}
+{"name":"faucibus","favorite_number":9,"favorite_color":"Purple"}
+{"name":"integer","favorite_number":77,"favorite_color":"Orange"}
+{"name":"lectus","favorite_number":6,"favorite_color":"Pink"}
+{"name":"sed","favorite_number":39,"favorite_color":"Maroon"}
+{"name":"vivamus","favorite_number":42,"favorite_color":"Goldenrod"}
+{"name":"quam","favorite_number":26,"favorite_color":"Pink"}
+{"name":"massa","favorite_number":23,"favorite_color":"Yellow"}
+{"name":"massa","favorite_number":59,"favorite_color":"Mauv"}
+{"name":"porttitor","favorite_number":39,"favorite_color":"Indigo"}
+{"name":"nec","favorite_number":19,"favorite_color":"Indigo"}
+{"name":"viverra","favorite_number":88,"favorite_color":"Teal"}
+{"name":"ultrices","favorite_number":40,"favorite_color":"Turquoise"}
+{"name":"non","favorite_number":24,"favorite_color":"Orange"}
+{"name":"praesent","favorite_number":61,"favorite_color":"Puce"}
+{"name":"in","favorite_number":27,"favorite_color":"Violet"}
+{"name":"amet","favorite_number":6,"favorite_color":"Violet"}
+{"name":"eu","favorite_number":82,"favorite_color":"Turquoise"}
+{"name":"dolor","favorite_number":3,"favorite_color":"Teal"}
+{"name":"est","favorite_number":4,"favorite_color":"Green"}
+{"name":"venenatis","favorite_number":21,"favorite_color":"Blue"}
+{"name":"dignissim","favorite_number":89,"favorite_color":"Fuscia"}
+{"name":"ac","favorite_number":85,"favorite_color":"Green"}
+{"name":"ac","favorite_number":40,"favorite_color":"Indigo"}
+{"name":"imperdiet","favorite_number":50,"favorite_color":"Green"}
+{"name":"dapibus","favorite_number":15,"favorite_color":"Teal"}
+{"name":"purus","favorite_number":31,"favorite_color":"Crimson"}
+{"name":"at","favorite_number":78,"favorite_color":"Yellow"}
+{"name":"donec","favorite_number":60,"favorite_color":"Maroon"}
+{"name":"in","favorite_number":19,"favorite_color":"Maroon"}
+{"name":"natoque","favorite_number":23,"favorite_color":"Red"}
+{"name":"morbi","favorite_number":31,"favorite_color":"Indigo"}
+{"name":"tristique","favorite_number":25,"favorite_color":"Purple"}
+{"name":"magnis","favorite_number":4,"favorite_color":"Red"}
+{"name":"nibh","favorite_number":9,"favorite_color":"Orange"}
+{"name":"orci","favorite_number":60,"favorite_color":"Khaki"}
+{"name":"cras","favorite_number":91,"favorite_color":"Green"}
+{"name":"viverra","favorite_number":58,"favorite_color":"Red"}
+{"name":"nulla","favorite_number":52,"favorite_color":"Aquamarine"}
+{"name":"nam","favorite_number":33,"favorite_color":"Maroon"}
+{"name":"justo","favorite_number":90,"favorite_color":"Violet"}
+{"name":"interdum","favorite_number":51,"favorite_color":"Orange"}
+{"name":"ac","favorite_number":34,"favorite_color":"Purple"}
+{"name":"mattis","favorite_number":77,"favorite_color":"Khaki"}
+{"name":"ligula","favorite_number":10,"favorite_color":"Orange"}
+{"name":"vel","favorite_number":87,"favorite_color":"Red"}
+{"name":"venenatis","favorite_number":51,"favorite_color":"Yellow"}
+{"name":"orci","favorite_number":71,"favorite_color":"Fuscia"}
+{"name":"mattis","favorite_number":9,"favorite_color":"Violet"}
+{"name":"semper","favorite_number":53,"favorite_color":"Goldenrod"}
+{"name":"vel","favorite_number":9,"favorite_color":"Crimson"}
+{"name":"velit","favorite_number":87,"favorite_color":"Turquoise"}
+{"name":"metus","favorite_number":11,"favorite_color":"Violet"}
+{"name":"tincidunt","favorite_number":19,"favorite_color":"Orange"}
+{"name":"elit","favorite_number":46,"favorite_color":"Pink"}
+{"name":"ante","favorite_number":11,"favorite_color":"Indigo"}
+{"name":"semper","favorite_number":70,"favorite_color":"Turquoise"}
+{"name":"pharetra","favorite_number":65,"favorite_color":"Red"}
+{"name":"lorem","favorite_number":71,"favorite_color":"Puce"}
+{"name":"ullamcorper","favorite_number":45,"favorite_color":"Fuscia"}
+{"name":"proin","favorite_number":12,"favorite_color":"Teal"}
+{"name":"vulputate","favorite_number":20,"favorite_color":"Goldenrod"}
+{"name":"justo","favorite_number":73,"favorite_color":"Green"}
+{"name":"vestibulum","favorite_number":17,"favorite_color":"Orange"}
+{"name":"sapien","favorite_number":31,"favorite_color":"Mauv"}
+{"name":"tempor","favorite_number":93,"favorite_color":"Orange"}
+{"name":"orci","favorite_number":46,"favorite_color":"Turquoise"}
+{"name":"faucibus","favorite_number":62,"favorite_color":"Green"}
+{"name":"nonummy","favorite_number":77,"favorite_color":"Aquamarine"}
+{"name":"suspendisse","favorite_number":95,"favorite_color":"Maroon"}
+{"name":"urna","favorite_number":85,"favorite_color":"Purple"}
+{"name":"turpis","favorite_number":80,"favorite_color":"Khaki"}
+{"name":"vestibulum","favorite_number":67,"favorite_color":"Turquoise"}
+{"name":"platea","favorite_number":65,"favorite_color":"Orange"}
+{"name":"augue","favorite_number":75,"favorite_color":"Orange"}
+{"name":"id","favorite_number":24,"favorite_color":"Orange"}
+{"name":"sapien","favorite_number":34,"favorite_color":"Teal"}
+{"name":"platea","favorite_number":20,"favorite_color":"Green"}
+{"name":"ut","favorite_number":22,"favorite_color":"Pink"}
+{"name":"varius","favorite_number":87,"favorite_color":"Orange"}
+{"name":"consequat","favorite_number":2,"favorite_color":"Teal"}
+{"name":"lacus","favorite_number":22,"favorite_color":"Indigo"}
+{"name":"aliquam","favorite_number":96,"favorite_color":"Turquoise"}
+{"name":"nisl","favorite_number":4,"favorite_color":"Orange"}
+{"name":"sit","favorite_number":50,"favorite_color":"Turquoise"}
+{"name":"congue","favorite_number":26,"favorite_color":"Green"}
+{"name":"cursus","favorite_number":5,"favorite_color":"Puce"}
+{"name":"tellus","favorite_number":40,"favorite_color":"Indigo"}
+{"name":"ultrices","favorite_number":80,"favorite_color":"Crimson"}
+{"name":"erat","favorite_number":65,"favorite_color":"Goldenrod"}
+{"name":"non","favorite_number":62,"favorite_color":"Puce"}
+{"name":"ligula","favorite_number":94,"favorite_color":"Yellow"}
+{"name":"sem","favorite_number":94,"favorite_color":"Red"}
+{"name":"quam","favorite_number":96,"favorite_color":"Maroon"}
+{"name":"consequat","favorite_number":21,"favorite_color":"Yellow"}
+{"name":"nulla","favorite_number":60,"favorite_color":"Fuscia"}
+{"name":"nibh","favorite_number":47,"favorite_color":"Maroon"}
+{"name":"accumsan","favorite_number":4,"favorite_color":"Puce"}
+{"name":"rutrum","favorite_number":64,"favorite_color":"Fuscia"}
+{"name":"mauris","favorite_number":59,"favorite_color":"Blue"}
+{"name":"proin","favorite_number":28,"favorite_color":"Khaki"}
+{"name":"rutrum","favorite_number":81,"favorite_color":"Green"}
+{"name":"imperdiet","favorite_number":87,"favorite_color":"Teal"}
+{"name":"penatibus","favorite_number":45,"favorite_color":"Blue"}
+{"name":"mattis","favorite_number":97,"favorite_color":"Purple"}
+{"name":"sed","favorite_number":65,"favorite_color":"Puce"}
+{"name":"vel","favorite_number":16,"favorite_color":"Goldenrod"}
+{"name":"consectetuer","favorite_number":29,"favorite_color":"Crimson"}
+{"name":"neque","favorite_number":66,"favorite_color":"Khaki"}
+{"name":"nibh","favorite_number":58,"favorite_color":"Purple"}
+{"name":"tempus","favorite_number":84,"favorite_color":"Maroon"}
+{"name":"aliquet","favorite_number":79,"favorite_color":"Green"}
+{"name":"tempus","favorite_number":3,"favorite_color":"Teal"}
+{"name":"platea","favorite_number":34,"favorite_color":"Green"}
+{"name":"curae","favorite_number":88,"favorite_color":"Green"}
+{"name":"ut","favorite_number":18,"favorite_color":"Blue"}
+{"name":"vel","favorite_number":88,"favorite_color":"Mauv"}
+{"name":"quis","favorite_number":50,"favorite_color":"Orange"}
+{"name":"ante","favorite_number":80,"favorite_color":"Teal"}
+{"name":"vestibulum","favorite_number":14,"favorite_color":"Violet"}
+{"name":"erat","favorite_number":1,"favorite_color":"Indigo"}
+{"name":"eu","favorite_number":44,"favorite_color":"Violet"}
+{"name":"lectus","favorite_number":30,"favorite_color":"Teal"}
+{"name":"sit","favorite_number":53,"favorite_color":"Pink"}
+{"name":"at","favorite_number":40,"favorite_color":"Teal"}
+{"name":"molestie","favorite_number":56,"favorite_color":"Green"}
+{"name":"urna","favorite_number":64,"favorite_color":"Crimson"}
+{"name":"viverra","favorite_number":9,"favorite_color":"Violet"}
+{"name":"morbi","favorite_number":81,"favorite_color":"Orange"}
+{"name":"neque","favorite_number":66,"favorite_color":"Yellow"}
+{"name":"nunc","favorite_number":49,"favorite_color":"Khaki"}
+{"name":"vestibulum","favorite_number":47,"favorite_color":"Green"}
+{"name":"posuere","favorite_number":60,"favorite_color":"Indigo"}
+{"name":"luctus","favorite_number":88,"favorite_color":"Purple"}
+{"name":"platea","favorite_number":89,"favorite_color":"Violet"}
+{"name":"aliquet","favorite_number":1,"favorite_color":"Goldenrod"}
+{"name":"pede","favorite_number":42,"favorite_color":"Pink"}
+{"name":"nibh","favorite_number":81,"favorite_color":"Orange"}
+{"name":"fusce","favorite_number":4,"favorite_color":"Aquamarine"}
+{"name":"eu","favorite_number":2,"favorite_color":"Turquoise"}
+{"name":"suspendisse","favorite_number":72,"favorite_color":"Mauv"}
+{"name":"proin","favorite_number":59,"favorite_color":"Maroon"}
+{"name":"erat","favorite_number":36,"favorite_color":"Aquamarine"}
+{"name":"pulvinar","favorite_number":11,"favorite_color":"Khaki"}
+{"name":"tristique","favorite_number":28,"favorite_color":"Purple"}
+{"name":"nulla","favorite_number":68,"favorite_color":"Purple"}
+{"name":"sollicitudin","favorite_number":46,"favorite_color":"Maroon"}
+{"name":"sed","favorite_number":34,"favorite_color":"Fuscia"}
+{"name":"vehicula","favorite_number":61,"favorite_color":"Maroon"}
+{"name":"cursus","favorite_number":92,"favorite_color":"Goldenrod"}
+{"name":"viverra","favorite_number":68,"favorite_color":"Maroon"}
+{"name":"ullamcorper","favorite_number":49,"favorite_color":"Goldenrod"}
+{"name":"consequat","favorite_number":39,"favorite_color":"Violet"}
+{"name":"proin","favorite_number":50,"favorite_color":"Khaki"}
+{"name":"vel","favorite_number":49,"favorite_color":"Turquoise"}
+{"name":"aliquam","favorite_number":31,"favorite_color":"Aquamarine"}
+{"name":"vestibulum","favorite_number":31,"favorite_color":"Violet"}
+{"name":"pede","favorite_number":48,"favorite_color":"Orange"}
+{"name":"at","favorite_number":7,"favorite_color":"Goldenrod"}
+{"name":"eu","favorite_number":14,"favorite_color":"Turquoise"}
+{"name":"congue","favorite_number":90,"favorite_color":"Pink"}
+{"name":"risus","favorite_number":77,"favorite_color":"Pink"}
+{"name":"elit","favorite_number":7,"favorite_color":"Fuscia"}
+{"name":"dictumst","favorite_number":45,"favorite_color":"Blue"}
+{"name":"at","favorite_number":1,"favorite_color":"Orange"}
+{"name":"mauris","favorite_number":50,"favorite_color":"Red"}
+{"name":"et","favorite_number":65,"favorite_color":"Khaki"}
+{"name":"at","favorite_number":71,"favorite_color":"Goldenrod"}
+{"name":"non","favorite_number":80,"favorite_color":"Orange"}
+{"name":"eleifend","favorite_number":11,"favorite_color":"Crimson"}
+{"name":"at","favorite_number":99,"favorite_color":"Crimson"}
+{"name":"libero","favorite_number":53,"favorite_color":"Fuscia"}
+{"name":"massa","favorite_number":51,"favorite_color":"Teal"}
+{"name":"cursus","favorite_number":25,"favorite_color":"Orange"}
+{"name":"ultrices","favorite_number":92,"favorite_color":"Orange"}
+{"name":"sollicitudin","favorite_number":26,"favorite_color":"Maroon"}
+{"name":"consequat","favorite_number":31,"favorite_color":"Blue"}
+{"name":"luctus","favorite_number":97,"favorite_color":"Yellow"}
+{"name":"vivamus","favorite_number":39,"favorite_color":"Pink"}
+{"name":"lorem","favorite_number":87,"favorite_color":"Blue"}
+{"name":"porta","favorite_number":16,"favorite_color":"Violet"}
+{"name":"rutrum","favorite_number":12,"favorite_color":"Pink"}
+{"name":"in","favorite_number":90,"favorite_color":"Yellow"}
+{"name":"donec","favorite_number":88,"favorite_color":"Pink"}
+{"name":"aliquet","favorite_number":36,"favorite_color":"Aquamarine"}
+{"name":"leo","favorite_number":87,"favorite_color":"Purple"}
+{"name":"nisi","favorite_number":61,"favorite_color":"Fuscia"}
+{"name":"aenean","favorite_number":59,"favorite_color":"Pink"}
+{"name":"et","favorite_number":27,"favorite_color":"Teal"}
+{"name":"in","favorite_number":78,"favorite_color":"Green"}
+{"name":"sem","favorite_number":19,"favorite_color":"Pink"}
+{"name":"amet","favorite_number":40,"favorite_color":"Khaki"}
+{"name":"consequat","favorite_number":4,"favorite_color":"Orange"}
+{"name":"quam","favorite_number":88,"favorite_color":"Mauv"}
+{"name":"sed","favorite_number":35,"favorite_color":"Indigo"}
+{"name":"nullam","favorite_number":1,"favorite_color":"Yellow"}
+{"name":"posuere","favorite_number":71,"favorite_color":"Goldenrod"}
+{"name":"cras","favorite_number":5,"favorite_color":"Crimson"}
+{"name":"erat","favorite_number":58,"favorite_color":"Mauv"}
+{"name":"nulla","favorite_number":68,"favorite_color":"Purple"}
+{"name":"nulla","favorite_number":60,"favorite_color":"Khaki"}
+{"name":"vestibulum","favorite_number":53,"favorite_color":"Puce"}
+{"name":"ut","favorite_number":92,"favorite_color":"Fuscia"}
+{"name":"justo","favorite_number":38,"favorite_color":"Maroon"}
+{"name":"mattis","favorite_number":98,"favorite_color":"Aquamarine"}
+{"name":"in","favorite_number":28,"favorite_color":"Purple"}
+{"name":"bibendum","favorite_number":29,"favorite_color":"Mauv"}
+{"name":"lacinia","favorite_number":1,"favorite_color":"Green"}
+{"name":"velit","favorite_number":31,"favorite_color":"Goldenrod"}
+{"name":"in","favorite_number":12,"favorite_color":"Goldenrod"}
+{"name":"sem","favorite_number":5,"favorite_color":"Khaki"}
+{"name":"mauris","favorite_number":61,"favorite_color":"Khaki"}
+{"name":"eros","favorite_number":50,"favorite_color":"Turquoise"}
+{"name":"amet","favorite_number":65,"favorite_color":"Crimson"}
+{"name":"in","favorite_number":58,"favorite_color":"Fuscia"}
+{"name":"sed","favorite_number":36,"favorite_color":"Orange"}
+{"name":"vestibulum","favorite_number":2,"favorite_color":"Mauv"}
+{"name":"sem","favorite_number":45,"favorite_color":"Indigo"}
+{"name":"aenean","favorite_number":67,"favorite_color":"Fuscia"}
+{"name":"parturient","favorite_number":73,"favorite_color":"Pink"}
+{"name":"vitae","favorite_number":35,"favorite_color":"Purple"}
+{"name":"in","favorite_number":68,"favorite_color":"Purple"}
+{"name":"dictumst","favorite_number":22,"favorite_color":"Orange"}
+{"name":"cubilia","favorite_number":16,"favorite_color":"Yellow"}
+{"name":"nisl","favorite_number":83,"favorite_color":"Mauv"}
+{"name":"nam","favorite_number":43,"favorite_color":"Yellow"}
+{"name":"urna","favorite_number":97,"favorite_color":"Turquoise"}
+{"name":"quis","favorite_number":40,"favorite_color":"Crimson"}
+{"name":"hac","favorite_number":64,"favorite_color":"Teal"}
+{"name":"at","favorite_number":92,"favorite_color":"Pink"}
+{"name":"id","favorite_number":89,"favorite_color":"Maroon"}
+{"name":"integer","favorite_number":52,"favorite_color":"Pink"}
+{"name":"quis","favorite_number":63,"favorite_color":"Orange"}
+{"name":"dui","favorite_number":55,"favorite_color":"Red"}
+{"name":"lectus","favorite_number":66,"favorite_color":"Red"}
+{"name":"proin","favorite_number":56,"favorite_color":"Purple"}
+{"name":"lectus","favorite_number":64,"favorite_color":"Purple"}
+{"name":"sit","favorite_number":66,"favorite_color":"Indigo"}
+{"name":"integer","favorite_number":13,"favorite_color":"Green"}
+{"name":"at","favorite_number":15,"favorite_color":"Crimson"}
+{"name":"aliquet","favorite_number":59,"favorite_color":"Khaki"}
+{"name":"pretium","favorite_number":23,"favorite_color":"Orange"}
+{"name":"eleifend","favorite_number":62,"favorite_color":"Yellow"}
+{"name":"odio","favorite_number":89,"favorite_color":"Goldenrod"}
+{"name":"fringilla","favorite_number":8,"favorite_color":"Indigo"}
+{"name":"nulla","favorite_number":96,"favorite_color":"Maroon"}
+{"name":"ac","favorite_number":65,"favorite_color":"Crimson"}
+{"name":"tristique","favorite_number":57,"favorite_color":"Green"}
+{"name":"bibendum","favorite_number":59,"favorite_color":"Orange"}
+{"name":"at","favorite_number":42,"favorite_color":"Blue"}
+{"name":"sapien","favorite_number":16,"favorite_color":"Teal"}
+{"name":"ipsum","favorite_number":31,"favorite_color":"Turquoise"}
+{"name":"enim","favorite_number":65,"favorite_color":"Indigo"}
+{"name":"feugiat","favorite_number":76,"favorite_color":"Fuscia"}
+{"name":"gravida","favorite_number":33,"favorite_color":"Puce"}
+{"name":"gravida","favorite_number":37,"favorite_color":"Indigo"}
+{"name":"nulla","favorite_number":59,"favorite_color":"Maroon"}
+{"name":"sem","favorite_number":35,"favorite_color":"Violet"}
+{"name":"quis","favorite_number":19,"favorite_color":"Puce"}
+{"name":"morbi","favorite_number":45,"favorite_color":"Puce"}
+{"name":"posuere","favorite_number":12,"favorite_color":"Purple"}
+{"name":"semper","favorite_number":70,"favorite_color":"Green"}
+{"name":"mauris","favorite_number":22,"favorite_color":"Turquoise"}
+{"name":"nunc","favorite_number":27,"favorite_color":"Goldenrod"}
+{"name":"quis","favorite_number":22,"favorite_color":"Blue"}
+{"name":"integer","favorite_number":97,"favorite_color":"Puce"}
+{"name":"feugiat","favorite_number":1,"favorite_color":"Indigo"}
+{"name":"at","favorite_number":96,"favorite_color":"Blue"}
+{"name":"id","favorite_number":24,"favorite_color":"Crimson"}
+{"name":"odio","favorite_number":30,"favorite_color":"Fuscia"}
+{"name":"aenean","favorite_number":37,"favorite_color":"Orange"}
+{"name":"fusce","favorite_number":93,"favorite_color":"Goldenrod"}
+{"name":"dolor","favorite_number":65,"favorite_color":"Yellow"}
+{"name":"semper","favorite_number":92,"favorite_color":"Blue"}
+{"name":"morbi","favorite_number":31,"favorite_color":"Violet"}
+{"name":"eu","favorite_number":44,"favorite_color":"Purple"}
+{"name":"id","favorite_number":43,"favorite_color":"Indigo"}
+{"name":"in","favorite_number":80,"favorite_color":"Turquoise"}
+{"name":"dictumst","favorite_number":24,"favorite_color":"Indigo"}
+{"name":"magnis","favorite_number":3,"favorite_color":"Pink"}
+{"name":"in","favorite_number":21,"favorite_color":"Blue"}
+{"name":"neque","favorite_number":63,"favorite_color":"Green"}
+{"name":"dictumst","favorite_number":3,"favorite_color":"Orange"}
+{"name":"libero","favorite_number":81,"favorite_color":"Goldenrod"}
+{"name":"ultrices","favorite_number":84,"favorite_color":"Crimson"}
+{"name":"vestibulum","favorite_number":90,"favorite_color":"Goldenrod"}
+{"name":"in","favorite_number":49,"favorite_color":"Maroon"}
+{"name":"sed","favorite_number":91,"favorite_color":"Crimson"}
+{"name":"vestibulum","favorite_number":72,"favorite_color":"Aquamarine"}
+{"name":"vestibulum","favorite_number":89,"favorite_color":"Puce"}
+{"name":"augue","favorite_number":12,"favorite_color":"Pink"}
+{"name":"eu","favorite_number":4,"favorite_color":"Mauv"}
+{"name":"nulla","favorite_number":52,"favorite_color":"Maroon"}
+{"name":"posuere","favorite_number":3,"favorite_color":"Purple"}
+{"name":"duis","favorite_number":53,"favorite_color":"Purple"}
+{"name":"felis","favorite_number":81,"favorite_color":"Indigo"}
+{"name":"id","favorite_number":57,"favorite_color":"Red"}
+{"name":"lobortis","favorite_number":86,"favorite_color":"Aquamarine"}
+{"name":"ac","favorite_number":63,"favorite_color":"Khaki"}
+{"name":"nullam","favorite_number":26,"favorite_color":"Indigo"}
+{"name":"vestibulum","favorite_number":22,"favorite_color":"Purple"}
+{"name":"erat","favorite_number":67,"favorite_color":"Orange"}
+{"name":"sed","favorite_number":51,"favorite_color":"Indigo"}
+{"name":"sed","favorite_number":62,"favorite_color":"Violet"}
+{"name":"integer","favorite_number":78,"favorite_color":"Orange"}
+{"name":"ut","favorite_number":50,"favorite_color":"Aquamarine"}
+{"name":"congue","favorite_number":69,"favorite_color":"Teal"}
+{"name":"lobortis","favorite_number":100,"favorite_color":"Teal"}
+{"name":"odio","favorite_number":68,"favorite_color":"Blue"}
+{"name":"fringilla","favorite_number":65,"favorite_color":"Indigo"}
+{"name":"quam","favorite_number":76,"favorite_color":"Green"}
+{"name":"velit","favorite_number":9,"favorite_color":"Aquamarine"}
+{"name":"volutpat","favorite_number":43,"favorite_color":"Teal"}
+{"name":"cursus","favorite_number":55,"favorite_color":"Orange"}
+{"name":"nibh","favorite_number":44,"favorite_color":"Goldenrod"}
+{"name":"et","favorite_number":46,"favorite_color":"Khaki"}
+{"name":"nisi","favorite_number":65,"favorite_color":"Fuscia"}
+{"name":"consequat","favorite_number":47,"favorite_color":"Blue"}
+{"name":"nam","favorite_number":28,"favorite_color":"Mauv"}
+{"name":"porttitor","favorite_number":72,"favorite_color":"Orange"}
+{"name":"odio","favorite_number":95,"favorite_color":"Pink"}
+{"name":"tortor","favorite_number":37,"favorite_color":"Khaki"}
+{"name":"convallis","favorite_number":66,"favorite_color":"Goldenrod"}
+{"name":"ac","favorite_number":56,"favorite_color":"Red"}
+{"name":"porttitor","favorite_number":20,"favorite_color":"Orange"}
+{"name":"magnis","favorite_number":11,"favorite_color":"Khaki"}
+{"name":"dapibus","favorite_number":90,"favorite_color":"Teal"}
+{"name":"ac","favorite_number":80,"favorite_color":"Yellow"}
+{"name":"aliquam","favorite_number":25,"favorite_color":"Puce"}
+{"name":"sed","favorite_number":28,"favorite_color":"Aquamarine"}
+{"name":"eget","favorite_number":11,"favorite_color":"Red"}
+{"name":"leo","favorite_number":45,"favorite_color":"Yellow"}
+{"name":"pede","favorite_number":41,"favorite_color":"Blue"}
+{"name":"consequat","favorite_number":12,"favorite_color":"Puce"}
+{"name":"vulputate","favorite_number":7,"favorite_color":"Maroon"}
+{"name":"ridiculus","favorite_number":68,"favorite_color":"Puce"}
+{"name":"proin","favorite_number":79,"favorite_color":"Khaki"}
+{"name":"lobortis","favorite_number":99,"favorite_color":"Orange"}
+{"name":"erat","favorite_number":12,"favorite_color":"Blue"}
+{"name":"sed","favorite_number":66,"favorite_color":"Purple"}
+{"name":"congue","favorite_number":73,"favorite_color":"Puce"}
+{"name":"nunc","favorite_number":23,"favorite_color":"Turquoise"}
+{"name":"lobortis","favorite_number":50,"favorite_color":"Purple"}
+{"name":"sit","favorite_number":78,"favorite_color":"Turquoise"}
+{"name":"nonummy","favorite_number":1,"favorite_color":"Red"}
+{"name":"velit","favorite_number":12,"favorite_color":"Khaki"}
+{"name":"sed","favorite_number":79,"favorite_color":"Goldenrod"}
+{"name":"lobortis","favorite_number":19,"favorite_color":"Red"}
+{"name":"in","favorite_number":71,"favorite_color":"Orange"}
+{"name":"rutrum","favorite_number":2,"favorite_color":"Maroon"}
+{"name":"tincidunt","favorite_number":10,"favorite_color":"Maroon"}
+{"name":"ac","favorite_number":25,"favorite_color":"Blue"}
+{"name":"donec","favorite_number":100,"favorite_color":"Puce"}
+{"name":"feugiat","favorite_number":24,"favorite_color":"Turquoise"}
+{"name":"curae","favorite_number":36,"favorite_color":"Aquamarine"}
+{"name":"vel","favorite_number":86,"favorite_color":"Purple"}
+{"name":"viverra","favorite_number":30,"favorite_color":"Puce"}
+{"name":"amet","favorite_number":78,"favorite_color":"Crimson"}
+{"name":"consectetuer","favorite_number":87,"favorite_color":"Pink"}
+{"name":"mollis","favorite_number":19,"favorite_color":"Teal"}
+{"name":"nisl","favorite_number":98,"favorite_color":"Khaki"}
+{"name":"faucibus","favorite_number":59,"favorite_color":"Crimson"}
+{"name":"vel","favorite_number":77,"favorite_color":"Fuscia"}
+{"name":"congue","favorite_number":27,"favorite_color":"Green"}
+{"name":"non","favorite_number":33,"favorite_color":"Green"}
+{"name":"est","favorite_number":76,"favorite_color":"Pink"}
+{"name":"tincidunt","favorite_number":10,"favorite_color":"Orange"}
+{"name":"ultrices","favorite_number":60,"favorite_color":"Khaki"}
+{"name":"nonummy","favorite_number":25,"favorite_color":"Khaki"}
+{"name":"magna","favorite_number":47,"favorite_color":"Pink"}
+{"name":"ut","favorite_number":73,"favorite_color":"Red"}
+{"name":"sit","favorite_number":15,"favorite_color":"Teal"}
+{"name":"congue","favorite_number":52,"favorite_color":"Turquoise"}
+{"name":"venenatis","favorite_number":43,"favorite_color":"Puce"}
+{"name":"in","favorite_number":1,"favorite_color":"Crimson"}
+{"name":"egestas","favorite_number":23,"favorite_color":"Pink"}
+{"name":"dui","favorite_number":38,"favorite_color":"Pink"}
+{"name":"arcu","favorite_number":94,"favorite_color":"Indigo"}
+{"name":"rutrum","favorite_number":88,"favorite_color":"Maroon"}
+{"name":"nulla","favorite_number":56,"favorite_color":"Indigo"}
+{"name":"enim","favorite_number":84,"favorite_color":"Puce"}
+{"name":"felis","favorite_number":8,"favorite_color":"Blue"}
+{"name":"volutpat","favorite_number":15,"favorite_color":"Crimson"}
+{"name":"nunc","favorite_number":39,"favorite_color":"Green"}
+{"name":"augue","favorite_number":59,"favorite_color":"Crimson"}
+{"name":"pellentesque","favorite_number":100,"favorite_color":"Maroon"}
+{"name":"vehicula","favorite_number":87,"favorite_color":"Aquamarine"}
+{"name":"leo","favorite_number":52,"favorite_color":"Purple"}
+{"name":"eleifend","favorite_number":29,"favorite_color":"Maroon"}
+{"name":"nulla","favorite_number":85,"favorite_color":"Aquamarine"}
+{"name":"eu","favorite_number":20,"favorite_color":"Yellow"}
+{"name":"non","favorite_number":59,"favorite_color":"Orange"}
+{"name":"imperdiet","favorite_number":61,"favorite_color":"Mauv"}
+{"name":"semper","favorite_number":21,"favorite_color":"Puce"}
+{"name":"nulla","favorite_number":33,"favorite_color":"Turquoise"}
+{"name":"varius","favorite_number":85,"favorite_color":"Green"}
+{"name":"sed","favorite_number":6,"favorite_color":"Violet"}
+{"name":"id","favorite_number":66,"favorite_color":"Fuscia"}
+{"name":"nam","favorite_number":41,"favorite_color":"Pink"}
+{"name":"et","favorite_number":9,"favorite_color":"Aquamarine"}
+{"name":"leo","favorite_number":91,"favorite_color":"Green"}
+{"name":"etiam","favorite_number":18,"favorite_color":"Red"}
+{"name":"justo","favorite_number":25,"favorite_color":"Purple"}
+{"name":"pretium","favorite_number":50,"favorite_color":"Blue"}
+{"name":"primis","favorite_number":11,"favorite_color":"Puce"}
+{"name":"eget","favorite_number":15,"favorite_color":"Pink"}
+{"name":"consectetuer","favorite_number":28,"favorite_color":"Mauv"}
+{"name":"non","favorite_number":49,"favorite_color":"Purple"}
+{"name":"convallis","favorite_number":60,"favorite_color":"Fuscia"}
+{"name":"dui","favorite_number":53,"favorite_color":"Aquamarine"}
+{"name":"consequat","favorite_number":69,"favorite_color":"Yellow"}
+{"name":"nullam","favorite_number":47,"favorite_color":"Violet"}
+{"name":"luctus","favorite_number":48,"favorite_color":"Mauv"}
+{"name":"duis","favorite_number":94,"favorite_color":"Khaki"}
+{"name":"mattis","favorite_number":2,"favorite_color":"Mauv"}
+{"name":"blandit","favorite_number":93,"favorite_color":"Teal"}
+{"name":"nec","favorite_number":96,"favorite_color":"Violet"}
+{"name":"tellus","favorite_number":48,"favorite_color":"Khaki"}
+{"name":"tortor","favorite_number":76,"favorite_color":"Mauv"}
+{"name":"nisl","favorite_number":92,"favorite_color":"Maroon"}
+{"name":"sed","favorite_number":29,"favorite_color":"Mauv"}
+{"name":"id","favorite_number":94,"favorite_color":"Aquamarine"}
+{"name":"aenean","favorite_number":8,"favorite_color":"Teal"}
+{"name":"justo","favorite_number":63,"favorite_color":"Pink"}
+{"name":"lacinia","favorite_number":50,"favorite_color":"Turquoise"}
+{"name":"eget","favorite_number":59,"favorite_color":"Green"}
+{"name":"ut","favorite_number":66,"favorite_color":"Orange"}
+{"name":"nulla","favorite_number":53,"favorite_color":"Orange"}
+{"name":"lacinia","favorite_number":56,"favorite_color":"Blue"}
+{"name":"consequat","favorite_number":76,"favorite_color":"Khaki"}
+{"name":"id","favorite_number":32,"favorite_color":"Crimson"}
+{"name":"blandit","favorite_number":72,"favorite_color":"Blue"}
+{"name":"non","favorite_number":95,"favorite_color":"Indigo"}
+{"name":"scelerisque","favorite_number":35,"favorite_color":"Pink"}
+{"name":"eu","favorite_number":44,"favorite_color":"Green"}
+{"name":"elementum","favorite_number":78,"favorite_color":"Indigo"}
+{"name":"non","favorite_number":22,"favorite_color":"Maroon"}
+{"name":"eget","favorite_number":82,"favorite_color":"Green"}
+{"name":"suspendisse","favorite_number":64,"favorite_color":"Orange"}
+{"name":"curae","favorite_number":16,"favorite_color":"Red"}
+{"name":"ut","favorite_number":89,"favorite_color":"Khaki"}
+{"name":"est","favorite_number":59,"favorite_color":"Aquamarine"}
+{"name":"enim","favorite_number":1,"favorite_color":"Maroon"}
+{"name":"nulla","favorite_number":73,"favorite_color":"Maroon"}
+{"name":"condimentum","favorite_number":13,"favorite_color":"Fuscia"}
+{"name":"quis","favorite_number":82,"favorite_color":"Maroon"}
+{"name":"mauris","favorite_number":45,"favorite_color":"Aquamarine"}
+{"name":"molestie","favorite_number":74,"favorite_color":"Indigo"}
+{"name":"tempus","favorite_number":56,"favorite_color":"Violet"}
+{"name":"tincidunt","favorite_number":48,"favorite_color":"Maroon"}
+{"name":"lectus","favorite_number":30,"favorite_color":"Crimson"}
+{"name":"donec","favorite_number":8,"favorite_color":"Khaki"}
+{"name":"quam","favorite_number":75,"favorite_color":"Orange"}
+{"name":"ut","favorite_number":95,"favorite_color":"Khaki"}
+{"name":"eget","favorite_number":29,"favorite_color":"Crimson"}
+{"name":"nisl","favorite_number":65,"favorite_color":"Mauv"}
+{"name":"sed","favorite_number":6,"favorite_color":"Aquamarine"}
+{"name":"morbi","favorite_number":66,"favorite_color":"Green"}
+{"name":"sapien","favorite_number":42,"favorite_color":"Fuscia"}
+{"name":"blandit","favorite_number":12,"favorite_color":"Green"}
+{"name":"odio","favorite_number":33,"favorite_color":"Red"}
+{"name":"non","favorite_number":11,"favorite_color":"Teal"}
+{"name":"aenean","favorite_number":100,"favorite_color":"Goldenrod"}
+{"name":"elementum","favorite_number":66,"favorite_color":"Indigo"}
+{"name":"pede","favorite_number":47,"favorite_color":"Orange"}
+{"name":"id","favorite_number":30,"favorite_color":"Orange"}
+{"name":"interdum","favorite_number":4,"favorite_color":"Violet"}
+{"name":"in","favorite_number":33,"favorite_color":"Orange"}
+{"name":"ut","favorite_number":41,"favorite_color":"Blue"}
+{"name":"in","favorite_number":57,"favorite_color":"Turquoise"}
+{"name":"pharetra","favorite_number":37,"favorite_color":"Teal"}
+{"name":"curabitur","favorite_number":97,"favorite_color":"Red"}
+{"name":"nibh","favorite_number":49,"favorite_color":"Green"}
+{"name":"ipsum","favorite_number":65,"favorite_color":"Yellow"}
+{"name":"nisl","favorite_number":53,"favorite_color":"Purple"}
+{"name":"felis","favorite_number":64,"favorite_color":"Maroon"}
+{"name":"sit","favorite_number":60,"favorite_color":"Khaki"}
+{"name":"tristique","favorite_number":50,"favorite_color":"Violet"}
+{"name":"egestas","favorite_number":35,"favorite_color":"Pink"}
+{"name":"non","favorite_number":73,"favorite_color":"Violet"}
+{"name":"non","favorite_number":17,"favorite_color":"Yellow"}
+{"name":"mattis","favorite_number":86,"favorite_color":"Turquoise"}
+{"name":"tellus","favorite_number":68,"favorite_color":"Green"}
+{"name":"lorem","favorite_number":64,"favorite_color":"Turquoise"}
+{"name":"imperdiet","favorite_number":81,"favorite_color":"Teal"}
+{"name":"lectus","favorite_number":1,"favorite_color":"Maroon"}
+{"name":"ipsum","favorite_number":20,"favorite_color":"Yellow"}
+{"name":"eget","favorite_number":98,"favorite_color":"Red"}
+{"name":"ultrices","favorite_number":56,"favorite_color":"Mauv"}
+{"name":"adipiscing","favorite_number":20,"favorite_color":"Pink"}
+{"name":"morbi","favorite_number":26,"favorite_color":"Mauv"}
+{"name":"eleifend","favorite_number":43,"favorite_color":"Khaki"}
+{"name":"sit","favorite_number":88,"favorite_color":"Yellow"}
+{"name":"nulla","favorite_number":73,"favorite_color":"Maroon"}
+{"name":"condimentum","favorite_number":74,"favorite_color":"Teal"}
+{"name":"suspendisse","favorite_number":31,"favorite_color":"Blue"}
+{"name":"mi","favorite_number":77,"favorite_color":"Crimson"}
+{"name":"varius","favorite_number":53,"favorite_color":"Yellow"}
+{"name":"in","favorite_number":7,"favorite_color":"Red"}
+{"name":"nibh","favorite_number":99,"favorite_color":"Pink"}
+{"name":"nisi","favorite_number":9,"favorite_color":"Fuscia"}
+{"name":"enim","favorite_number":80,"favorite_color":"Mauv"}
+{"name":"id","favorite_number":29,"favorite_color":"Aquamarine"}
+{"name":"amet","favorite_number":67,"favorite_color":"Puce"}
+{"name":"massa","favorite_number":42,"favorite_color":"Khaki"}
+{"name":"posuere","favorite_number":41,"favorite_color":"Crimson"}
+{"name":"mauris","favorite_number":22,"favorite_color":"Goldenrod"}
+{"name":"nisl","favorite_number":37,"favorite_color":"Purple"}
+{"name":"nulla","favorite_number":21,"favorite_color":"Fuscia"}
+{"name":"integer","favorite_number":69,"favorite_color":"Puce"}
+{"name":"nam","favorite_number":13,"favorite_color":"Goldenrod"}
+{"name":"amet","favorite_number":49,"favorite_color":"Pink"}
+{"name":"metus","favorite_number":85,"favorite_color":"Blue"}
+{"name":"etiam","favorite_number":10,"favorite_color":"Puce"}
+{"name":"nec","favorite_number":69,"favorite_color":"Yellow"}
+{"name":"integer","favorite_number":78,"favorite_color":"Maroon"}
+{"name":"sit","favorite_number":43,"favorite_color":"Red"}
+{"name":"molestie","favorite_number":2,"favorite_color":"Khaki"}
+{"name":"ut","favorite_number":59,"favorite_color":"Yellow"}
+{"name":"viverra","favorite_number":81,"favorite_color":"Violet"}
+{"name":"velit","favorite_number":68,"favorite_color":"Violet"}
+{"name":"quam","favorite_number":89,"favorite_color":"Red"}
+{"name":"nibh","favorite_number":35,"favorite_color":"Turquoise"}
+{"name":"non","favorite_number":32,"favorite_color":"Teal"}
+{"name":"in","favorite_number":21,"favorite_color":"Turquoise"}
+{"name":"odio","favorite_number":11,"favorite_color":"Purple"}
+{"name":"in","favorite_number":60,"favorite_color":"Purple"}
+{"name":"magna","favorite_number":69,"favorite_color":"Red"}
+{"name":"sapien","favorite_number":9,"favorite_color":"Fuscia"}
+{"name":"congue","favorite_number":97,"favorite_color":"Goldenrod"}
+{"name":"nam","favorite_number":3,"favorite_color":"Puce"}
+{"name":"nunc","favorite_number":4,"favorite_color":"Maroon"}
+{"name":"id","favorite_number":4,"favorite_color":"Yellow"}
+{"name":"id","favorite_number":87,"favorite_color":"Pink"}
+{"name":"in","favorite_number":54,"favorite_color":"Crimson"}
+{"name":"dui","favorite_number":100,"favorite_color":"Pink"}
+{"name":"sed","favorite_number":43,"favorite_color":"Purple"}
+{"name":"dapibus","favorite_number":52,"favorite_color":"Fuscia"}
+{"name":"mauris","favorite_number":78,"favorite_color":"Khaki"}
+{"name":"cursus","favorite_number":28,"favorite_color":"Orange"}
+{"name":"nisi","favorite_number":62,"favorite_color":"Fuscia"}
+{"name":"justo","favorite_number":83,"favorite_color":"Turquoise"}
+{"name":"cursus","favorite_number":90,"favorite_color":"Khaki"}
+{"name":"ultrices","favorite_number":22,"favorite_color":"Turquoise"}
+{"name":"mi","favorite_number":99,"favorite_color":"Indigo"}
+{"name":"volutpat","favorite_number":98,"favorite_color":"Fuscia"}
+{"name":"faucibus","favorite_number":50,"favorite_color":"Goldenrod"}
+{"name":"eros","favorite_number":37,"favorite_color":"Violet"}
+{"name":"mi","favorite_number":87,"favorite_color":"Goldenrod"}
+{"name":"posuere","favorite_number":88,"favorite_color":"Fuscia"}
+{"name":"dui","favorite_number":77,"favorite_color":"Turquoise"}
+{"name":"est","favorite_number":65,"favorite_color":"Maroon"}
+{"name":"turpis","favorite_number":35,"favorite_color":"Blue"}
+{"name":"odio","favorite_number":48,"favorite_color":"Puce"}
+{"name":"pretium","favorite_number":21,"favorite_color":"Yellow"}
+{"name":"quis","favorite_number":45,"favorite_color":"Maroon"}
+{"name":"neque","favorite_number":7,"favorite_color":"Red"}
+{"name":"tortor","favorite_number":58,"favorite_color":"Khaki"}
+{"name":"molestie","favorite_number":56,"favorite_color":"Orange"}
+{"name":"vel","favorite_number":57,"favorite_color":"Fuscia"}
+{"name":"hac","favorite_number":75,"favorite_color":"Mauv"}
+{"name":"nisl","favorite_number":25,"favorite_color":"Violet"}
+{"name":"ultrices","favorite_number":25,"favorite_color":"Turquoise"}
+{"name":"nulla","favorite_number":22,"favorite_color":"Fuscia"}
+{"name":"condimentum","favorite_number":99,"favorite_color":"Crimson"}
+{"name":"tortor","favorite_number":6,"favorite_color":"Pink"}
+{"name":"mauris","favorite_number":19,"favorite_color":"Blue"}
+{"name":"vitae","favorite_number":82,"favorite_color":"Mauv"}
+{"name":"venenatis","favorite_number":96,"favorite_color":"Orange"}
+{"name":"parturient","favorite_number":61,"favorite_color":"Indigo"}
+{"name":"ligula","favorite_number":24,"favorite_color":"Yellow"}
+{"name":"bibendum","favorite_number":25,"favorite_color":"Teal"}
+{"name":"sodales","favorite_number":19,"favorite_color":"Purple"}
+{"name":"facilisi","favorite_number":37,"favorite_color":"Teal"}
+{"name":"lobortis","favorite_number":11,"favorite_color":"Goldenrod"}
+{"name":"suspendisse","favorite_number":12,"favorite_color":"Khaki"}
+{"name":"ligula","favorite_number":45,"favorite_color":"Goldenrod"}
+{"name":"id","favorite_number":24,"favorite_color":"Aquamarine"}
+{"name":"vestibulum","favorite_number":24,"favorite_color":"Pink"}
+{"name":"dapibus","favorite_number":34,"favorite_color":"Khaki"}
+{"name":"pellentesque","favorite_number":8,"favorite_color":"Crimson"}
+{"name":"ante","favorite_number":91,"favorite_color":"Yellow"}
+{"name":"quis","favorite_number":88,"favorite_color":"Red"}
+{"name":"est","favorite_number":16,"favorite_color":"Yellow"}
+{"name":"interdum","favorite_number":39,"favorite_color":"Khaki"}
+{"name":"eu","favorite_number":53,"favorite_color":"Yellow"}
+{"name":"suspendisse","favorite_number":73,"favorite_color":"Orange"}
+{"name":"augue","favorite_number":47,"favorite_color":"Green"}
+{"name":"ante","favorite_number":4,"favorite_color":"Goldenrod"}
+{"name":"odio","favorite_number":16,"favorite_color":"Purple"}
+{"name":"sed","favorite_number":57,"favorite_color":"Red"}
+{"name":"nibh","favorite_number":86,"favorite_color":"Goldenrod"}
+{"name":"libero","favorite_number":25,"favorite_color":"Indigo"}
+{"name":"libero","favorite_number":26,"favorite_color":"Crimson"}
+{"name":"ridiculus","favorite_number":5,"favorite_color":"Goldenrod"}
+{"name":"consectetuer","favorite_number":21,"favorite_color":"Green"}
+{"name":"quis","favorite_number":71,"favorite_color":"Red"}
+{"name":"ac","favorite_number":32,"favorite_color":"Fuscia"}
+{"name":"vestibulum","favorite_number":11,"favorite_color":"Fuscia"}
+{"name":"odio","favorite_number":80,"favorite_color":"Purple"}
+{"name":"augue","favorite_number":63,"favorite_color":"Orange"}
+{"name":"amet","favorite_number":74,"favorite_color":"Green"}
+{"name":"faucibus","favorite_number":2,"favorite_color":"Green"}
+{"name":"ac","favorite_number":13,"favorite_color":"Maroon"}
+{"name":"aliquet","favorite_number":81,"favorite_color":"Aquamarine"}
+{"name":"accumsan","favorite_number":7,"favorite_color":"Aquamarine"}
+{"name":"in","favorite_number":61,"favorite_color":"Yellow"}
+{"name":"blandit","favorite_number":100,"favorite_color":"Goldenrod"}
+{"name":"non","favorite_number":63,"favorite_color":"Goldenrod"}
+{"name":"ac","favorite_number":52,"favorite_color":"Indigo"}
+{"name":"consequat","favorite_number":27,"favorite_color":"Goldenrod"}
+{"name":"odio","favorite_number":95,"favorite_color":"Maroon"}
+{"name":"sapien","favorite_number":31,"favorite_color":"Goldenrod"}
+{"name":"sed","favorite_number":66,"favorite_color":"Purple"}
+{"name":"integer","favorite_number":29,"favorite_color":"Indigo"}
+{"name":"posuere","favorite_number":81,"favorite_color":"Orange"}
+{"name":"diam","favorite_number":22,"favorite_color":"Aquamarine"}
+{"name":"lorem","favorite_number":71,"favorite_color":"Purple"}
+{"name":"vestibulum","favorite_number":95,"favorite_color":"Pink"}
+{"name":"massa","favorite_number":88,"favorite_color":"Fuscia"}
+{"name":"luctus","favorite_number":28,"favorite_color":"Goldenrod"}
+{"name":"habitasse","favorite_number":31,"favorite_color":"Fuscia"}
+{"name":"vulputate","favorite_number":95,"favorite_color":"Yellow"}
diff --git a/hoodie-common/src/test/resources/simple-test.avro b/hoodie-common/src/test/resources/simple-test.avro
new file mode 100644
index 000000000..eea932977
--- /dev/null
+++ b/hoodie-common/src/test/resources/simple-test.avro
@@ -0,0 +1,10 @@
+{
+"namespace": "example.avro",
+ "type": "record",
+ "name": "User",
+ "fields": [
+ {"name": "name", "type": "string"},
+ {"name": "favorite_number", "type": "long"},
+ {"name": "favorite_color", "type": "string"}
+ ]
+}
diff --git a/hoodie-hive/pom.xml b/hoodie-hive/pom.xml
index 919d9e0bb..1a19adc4b 100644
--- a/hoodie-hive/pom.xml
+++ b/hoodie-hive/pom.xml
@@ -110,6 +110,13 @@
hoodie-common
${project.version}
+
+ com.uber.hoodie
+ hoodie-common
+ ${project.version}
+ tests
+ test
+
diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/TestUtil.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/TestUtil.java
index f07bc0c3f..f1e0561d4 100644
--- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/TestUtil.java
+++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/TestUtil.java
@@ -17,6 +17,8 @@
package com.uber.hoodie.hive.util;
import com.google.common.collect.Sets;
+import com.uber.hoodie.common.minicluster.HdfsTestService;
+import com.uber.hoodie.common.minicluster.ZookeeperTestService;
import com.uber.hoodie.hive.HoodieHiveConfiguration;
import com.uber.hoodie.hive.client.HoodieHiveClient;
import com.uber.hoodie.hive.model.HoodieDatasetReference;
diff --git a/hoodie-utilities/pom.xml b/hoodie-utilities/pom.xml
index d44fbf298..13091006c 100644
--- a/hoodie-utilities/pom.xml
+++ b/hoodie-utilities/pom.xml
@@ -85,6 +85,46 @@
test
+
+ org.apache.hadoop
+ hadoop-hdfs
+ tests
+
+
+
+ org.mortbay.jetty
+ *
+
+
+ javax.servlet.jsp
+ *
+
+
+ javax.servlet
+ *
+
+
+
+
+ org.apache.hadoop
+ hadoop-common
+ tests
+
+
+ org.mortbay.jetty
+ *
+
+
+ javax.servlet.jsp
+ *
+
+
+ javax.servlet
+ *
+
+
+
+
com.uber.hoodie
hoodie-hive
@@ -198,6 +238,8 @@
1.10.19
test
+
+
diff --git a/pom.xml b/pom.xml
index 5e81073d1..9c24c0057 100644
--- a/pom.xml
+++ b/pom.xml
@@ -250,6 +250,7 @@
**/*.sh
**/dependency-reduced-pom.xml
**/test/resources/*.avro
+ **/test/resources/*.data
**/test/resources/*.schema
**/test/resources/*.csv
@@ -300,6 +301,12 @@
${parquet.version}
+
+ org.apache.avro
+ avro-mapred
+ 1.7.7
+
+
com.google.guava