public static List<HoodieRecord> generateHoodieTestRecords(int from, int limit, Schema schema) throws IOException, URISyntaxException { List<IndexedRecord> records = generateTestRecords(from, limit); return records.stream() .map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, schema)) .map(p -> convertToHoodieRecords(p, UUID.randomUUID().toString(), "000/00/00")).collect( Collectors.toList()); }
public static List<HoodieRecord> generateHoodieTestRecordsWithoutHoodieMetadata(int from, int limit) throws IOException, URISyntaxException { List<IndexedRecord> iRecords = generateTestRecords(from, limit); return iRecords .stream() .map(r -> new HoodieRecord<>(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"), new HoodieAvroPayload(Optional.of((GenericRecord) r)))).collect(Collectors.toList()); }
public static List<IndexedRecord> generateHoodieTestRecords(int from, int limit) throws IOException, URISyntaxException { List<IndexedRecord> records = generateTestRecords(from, limit); String commitTime = HoodieActiveTimeline.createNewCommitTime(); Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); return records.stream().map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, hoodieFieldsSchema)).map(p -> { p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString()); p.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00"); p.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime); return p; }).collect(Collectors.toList()); }
@Test public void testAppendNotSupported() throws IOException, URISyntaxException, InterruptedException { // Use some fs like LocalFileSystem, that does not support appends Path localPartitionPath = new Path("file://" + partitionPath); FileSystem localFs = FSUtils.getFs(localPartitionPath.toString(), HoodieTestUtils.getDefaultHadoopConf()); Path testPath = new Path(localPartitionPath, "append_test"); localFs.mkdirs(testPath); // Some data & append two times. List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100); Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); for (int i = 0; i < 2; i++) { HoodieLogFormat.newWriterBuilder().onParentPath(testPath) .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive").overBaseCommit("") .withFs(localFs).build().appendBlock(dataBlock).close(); } // ensure there are two log file versions, with same data. FileStatus[] statuses = localFs.listStatus(testPath); assertEquals(2, statuses.length); }
@Test public void testBasicAppend() throws IOException, InterruptedException, URISyntaxException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100); Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size = writer.getCurrentSize(); assertTrue("We just wrote a block - size should be > 0", size > 0); assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match", size, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); }
private static HoodieLogFile generateLogData(Path parquetFilePath, boolean isLogSchemaSimple) throws IOException, InterruptedException, URISyntaxException { Schema schema = (isLogSchemaSimple ? SchemaTestUtil.getSimpleSchema() : SchemaTestUtil.getEvolvedSchema()); HoodieDataFile dataFile = new HoodieDataFile(fileSystem.getFileStatus(parquetFilePath)); // Write a log file for this parquet file Writer logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(parquetFilePath.getParent()) .withFileExtension(HoodieLogFile.DELTA_EXTENSION) .withFileId(dataFile.getFileId()) .overBaseCommit(dataFile.getCommitTime()).withFs(fileSystem).build(); List<IndexedRecord> records = (isLogSchemaSimple ? SchemaTestUtil.generateTestRecords(0, 100) : SchemaTestUtil.generateEvolvedTestRecords(100, 100)); Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, dataFile.getCommitTime()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); logWriter.appendBlock(dataBlock); logWriter.close(); return logWriter.getLogFile(); }
@SuppressWarnings({"unchecked", "deprecation"}) private static void generateParquetData(Path filePath, boolean isParquetSchemaSimple) throws IOException, URISyntaxException, InterruptedException { Schema schema = (isParquetSchemaSimple ? SchemaTestUtil.getSimpleSchema() : SchemaTestUtil.getEvolvedSchema()); org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema); BloomFilter filter = new BloomFilter(1000, 0.0001); HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, filter); ParquetWriter writer = new ParquetWriter(filePath, writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION, fileSystem.getConf()); List<IndexedRecord> testRecords = (isParquetSchemaSimple ? SchemaTestUtil .generateTestRecords(0, 100) : SchemaTestUtil.generateEvolvedTestRecords(100, 100)); testRecords.forEach(s -> { try { writer.write(s); } catch (IOException e) { fail("IOException while writing test records as parquet" + e.toString()); } }); writer.close(); }
@SuppressWarnings("unchecked") @Test public void testBasicWriteAndScan() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); Schema schema = getSimpleSchema(); List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> copyOfRecords = records.stream().map( record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); writer.close(); Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); assertTrue("We wrote a block, we should be able to read it", reader.hasNext()); HoodieLogBlock nextBlock = reader.next(); assertEquals("The next block should be a data block", HoodieLogBlockType.AVRO_DATA_BLOCK, nextBlock.getBlockType()); HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock; assertEquals("Read records size should be equal to the written records size", copyOfRecords.size(), dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords, dataBlockRead.getRecords()); reader.close(); }
@Test public void testLeaseRecovery() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100); Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size1 = writer.getCurrentSize(); // do not close this writer - this simulates a data note appending to a log dying without closing the file // writer.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size2 = writer.getCurrentSize(); assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1); assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match", size2, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); }
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List<IndexedRecord> records1 = SchemaTestUtil.generateTestRecords(0, 100); Schema schema = getSimpleSchema(); List<IndexedRecord> copyOfRecords1 = records1.stream().map( .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") .withFs(fs).build(); List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> copyOfRecords2 = records2.stream().map( record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") .withFs(fs).build(); List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> copyOfRecords3 = records3.stream().map( record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
@Test public void testRollover() throws IOException, InterruptedException, URISyntaxException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100); Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); // Write out a block writer = writer.appendBlock(dataBlock); // Get the size of the block long size = writer.getCurrentSize(); writer.close(); // Create a writer with the size threshold as the size we just wrote - so this has to roll writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") .withFs(fs).withSizeThreshold(size - 1).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); assertEquals("This should be a new log file and hence size should be 0", 0, writer.getCurrentSize()); assertEquals("Version should be rolled to 2", 2, writer.getLogFile().getLogVersion()); writer.close(); }
.overBaseCommit("100").withFs(fs).build(); Schema schema = getSimpleSchema(); List<IndexedRecord> records1 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> copyOfRecords1 = records1.stream().map( record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") .withFs(fs).build(); List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> copyOfRecords2 = records2.stream().map( record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") .withFs(fs).build(); List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> copyOfRecords3 = records3.stream().map( record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
.overBaseCommit("100").withFs(fs).build(); Schema schema = getSimpleSchema(); List<IndexedRecord> records1 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> copyOfRecords1 = records1.stream().map( record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") .withFs(fs).build(); List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> copyOfRecords2 = records2.stream().map( record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") .withFs(fs).build(); List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100); List<IndexedRecord> copyOfRecords3 = records3.stream().map( record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100); Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header);
.overBaseCommit("100").withFs(fs).build(); Schema schema = getSimpleSchema(); List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100); Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock);
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 10); Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100); Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header);