1
0

[HUDI-232] Implement sealing/unsealing for HoodieRecord class (#938)

This commit is contained in:
leesf
2019-10-08 01:56:46 +08:00
committed by vinoth chandar
parent 8a55938ca1
commit d050d98071
11 changed files with 117 additions and 0 deletions

View File

@@ -0,0 +1,74 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import static org.junit.Assert.fail;
import java.util.List;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SchemaTestUtil;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
/**
* Tests for {@link HoodieRecord}.
*/
public class TestHoodieRecord {
private HoodieRecord hoodieRecord;
@Before
public void setUp() throws Exception {
final List<IndexedRecord> indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
final List<HoodieRecord> hoodieRecords = indexedRecords.stream()
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
new AvroBinaryTestPayload(Option.of((GenericRecord) r)))).collect(Collectors.toList());
hoodieRecord = hoodieRecords.get(0);
}
@Test
public void testModificationAfterSeal() {
hoodieRecord.seal();
final HoodieRecordLocation location = new HoodieRecordLocation("100", "0");
try {
hoodieRecord.setCurrentLocation(location);
fail("should fail since modification after sealed is not allowed");
} catch (Exception e) {
Assert.assertTrue(e instanceof UnsupportedOperationException);
}
}
@Test
public void testNormalModification() {
hoodieRecord.unseal();
final HoodieRecordLocation location = new HoodieRecordLocation("100", "0");
hoodieRecord.setCurrentLocation(location);
hoodieRecord.seal();
hoodieRecord.unseal();
hoodieRecord.setNewLocation(location);
hoodieRecord.seal();
}
}

View File

@@ -45,7 +45,9 @@ public class SpillableMapTestUtils {
recordKeys.add(key);
HoodieRecord record = new HoodieRecord<>(new HoodieKey(key, partitionPath),
new HoodieAvroPayload(Option.of((GenericRecord) r)));
record.unseal();
record.setCurrentLocation(new HoodieRecordLocation("DUMMY_COMMIT_TIME", "DUMMY_FILE_ID"));
record.seal();
records.put(key, record);
});
return recordKeys;