Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 0 additions & 1 deletion hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@ public class HoodieCLI {
public static HoodieTableMetaClient tableMetadata;
public static HoodieTableMetaClient syncTableMetadata;


public enum CLIState {
INIT, DATASET, SYNC
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,6 @@ public String deduplicate(
return "Deduplication failed ";
}


@CliCommand(value = "repair addpartitionmeta", help = "Add partition metadata to a dataset, if not present")
public String addPartitionMeta(
@CliOption(key = {"dryrun"}, help = "Should we actually add or just print what would be done",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,6 @@ public boolean isRefreshAvailable() {
return HoodieCLI.tableMetadata != null;
}


@CliAvailabilityIndicator({"savepoint create"})
public boolean isCreateSavepointAvailable() {
return HoodieCLI.tableMetadata != null;
Expand Down Expand Up @@ -127,7 +126,6 @@ public String rollbackToSavepoint(
return "Savepoint " + commitTime + " rolled back";
}


@CliCommand(value = "savepoints refresh", help = "Refresh the savepoints")
public String refreshMetaClient() throws IOException {
HoodieCLI.refreshTableMetadata();
Expand All @@ -140,5 +138,4 @@ private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String
return new HoodieWriteClient(jsc, config, false);
}


}
Original file line number Diff line number Diff line change
Expand Up @@ -57,5 +57,4 @@ public static void captureOutput(Process p) {
stdout.start();
}


}
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,6 @@ public Builder fromProperties(Properties props) {
return this;
}


public Builder withAutoClean(Boolean autoClean) {
props.setProperty(AUTO_CLEAN_PROP, String.valueOf(autoClean));
return this;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,6 @@ public Builder fromProperties(Properties props) {
return this;
}


public Builder on(boolean metricsOn) {
props.setProperty(METRICS_ON, String.valueOf(metricsOn));
return this;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,6 @@ public int getRollbackParallelism() {
return Integer.parseInt(props.getProperty(ROLLBACK_PARALLELISM));
}


public int getWriteBufferLimitBytes() {
return Integer.parseInt(props.getProperty(WRITE_BUFFER_LIMIT_BYTES, DEFAULT_WRITE_BUFFER_LIMIT_BYTES));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

package org.apache.hudi.exception;


/**
* <p>
* Exception thrown when dependent system is not available
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.function.Function2;


/**
* Map function that handles a sorted stream of HoodieRecords
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,6 @@ public LazyIterableIterator(Iterator<I> in) {
*/
protected abstract O computeNext();


/**
* Called once, after all elements are processed.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,6 @@ protected HoodieIndex(HoodieWriteConfig config) {
this.config = config;
}


public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(HoodieWriteConfig config,
JavaSparkContext jsc) throws HoodieIndexException {
switch (config.getIndexType()) {
Expand Down Expand Up @@ -108,7 +107,6 @@ public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeSt
*/
public abstract boolean canIndexLogFiles();


/**
* An index is "implicit" with respect to storage, if just writing new data to a file slice, updates the index as
* well. This is used by storage, to save memory footprint in certain cases.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,6 @@
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.Function2;


/**
* Hoodie Index implementation backed by an in-memory Hash map.
* <p>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -268,7 +268,6 @@ List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitio
}
}


@Override
public boolean rollbackCommit(String commitTime) {
// Nope, don't need to do anything.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -170,7 +170,6 @@ private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath)
return deletePaths;
}


/**
* Selects the versions for file for cleaning, such that it
* <p>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;


public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {

protected final String instantTime;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,6 @@ public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTabl
dataFileToBeMerged);
}


public static Schema createHoodieWriteSchema(Schema originalSchema) {
return HoodieAvroUtils.addMetadataFields(originalSchema);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
* limitations under the License.
*/


package org.apache.hudi.io.compact.strategy;

import com.google.common.annotations.VisibleForTesting;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,6 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
private final String commitTime;
private final Schema schema;


public HoodieParquetWriter(String commitTime, Path file, HoodieParquetConfig parquetConfig, Schema schema)
throws IOException {
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,6 @@
import org.apache.spark.api.java.function.PairFlatMapFunction;
import scala.Tuple2;


/**
* Implementation of a very heavily read-optimized Hoodie Table where
* <p>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -212,7 +212,6 @@ private Map<FileStatus, Boolean> deleteCleanedFiles(HoodieTableMetaClient metaCl
return results;
}


private Map<HeaderMetadataType, String> generateHeader(String commit) {
// generate metadata
Map<HeaderMetadataType, String> header = Maps.newHashMap();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,6 @@ public class WorkloadProfile<T extends HoodieRecordPayload> implements Serializa

private final WorkloadStat globalStat;


public WorkloadProfile(JavaRDD<HoodieRecord<T>> taggedRecords) {
this.taggedRecords = taggedRecords;
this.partitionPathStatMap = new HashMap<>();
Expand Down
1 change: 0 additions & 1 deletion hudi-client/src/test/java/HoodieClientExample.java
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,6 @@ public static void main(String[] args) throws Exception {
cli.run();
}


public void run() throws Exception {

SparkConf sparkConf = new SparkConf().setAppName("hoodie-client-example");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,6 @@ public void testCheckFailingAppear() throws Exception {
.asList(basePath + "/partition/path/f1_1-0-2_000.parquet", basePath + "/partition/path/f2_1-0-2_000.parquet"));
}


@Test(expected = TimeoutException.class)
public void testCheckFailingAppears() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,6 @@ private static void fakeMetaFile(String basePath, String commitTime, String suff
new File(parentPath + "/" + commitTime + suffix).createNewFile();
}


public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTimeline.COMMIT_EXTENSION);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,6 @@ public String getPartitionPath() {
return partitionPath;
}


@Override
public TestRawTripPayload preCombine(TestRawTripPayload another) {
return another;
Expand Down Expand Up @@ -129,7 +128,6 @@ private byte[] compressData(String jsonData) throws IOException {
return baos.toByteArray();
}


private String unCompressData(byte[] data) throws IOException {
try (InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data))) {
return FileIOUtils.readAsUTFString(iis, dataSize);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -291,7 +291,6 @@ public void testTagLocationWithEmptyRDD() throws Exception {
}
}


@Test
public void testTagLocation() throws Exception {
// We have some records to be tagged (two different partitions)
Expand Down Expand Up @@ -433,7 +432,6 @@ public void testCheckExists() throws Exception {
}
}


@Test
public void testBloomFilterFalseError() throws IOException, InterruptedException {
// We have two hoodie records
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -196,7 +196,6 @@ public void testExplodeRecordRDDWithFileComparisons() {
assertEquals(new HashSet<>(Arrays.asList("f4", "f1")), new HashSet<>(recordKeyToFileComps.get("005")));
}


@Test
public void testTagLocation() throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -250,7 +250,6 @@ public void testUpdateRecords() throws Exception {
assertEquals(4, writeStatus.getStat().getNumWrites());// 3 rewritten records + 1 new record
}


private List<HoodieRecord> newHoodieRecords(int n, String time) throws Exception {
List<HoodieRecord> records = new ArrayList<>();
for (int i = 0; i < n; i++) {
Expand Down Expand Up @@ -387,7 +386,6 @@ public void testFileSizeUpsertRecords() throws Exception {
assertEquals("If the number of records are more than 1150, then there should be a new file", 3, counts);
}


private UpsertPartitioner getUpsertPartitioner(int smallFileSize, int numInserts, int numUpdates, int fileSize,
String testPartitionPath, boolean autoSplitInserts) throws Exception {
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
Expand Down Expand Up @@ -419,7 +417,6 @@ private UpsertPartitioner getUpsertPartitioner(int smallFileSize, int numInserts
return partitioner;
}


@Test
public void testUpsertPartitioner() throws Exception {
final String testPartitionPath = "2016/09/26";
Expand All @@ -429,7 +426,6 @@ public void testUpsertPartitioner() throws Exception {
assertEquals("Total of 2 insert buckets", 2, insertBuckets.size());
}


@Test
public void testUpsertPartitionerWithSmallInsertHandling() throws Exception {
final String testPartitionPath = "2016/09/26";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,6 @@ public class HoodieAvroWriteSupport extends AvroWriteSupport {
public static final String HOODIE_MIN_RECORD_KEY_FOOTER = "hoodie_min_record_key";
public static final String HOODIE_MAX_RECORD_KEY_FOOTER = "hoodie_max_record_key";


public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, BloomFilter bloomFilter) {
super(schema, avroSchema);
this.bloomFilter = bloomFilter;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,6 @@ private byte[] compressData(String jsonData) throws IOException {
return baos.toByteArray();
}


private String unCompressData(byte[] data) throws IOException {
InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,6 @@ public class HoodiePartitionMetadata {

private static Logger log = LogManager.getLogger(HoodiePartitionMetadata.class);


/**
* Construct metadata from existing partition
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,7 +98,6 @@ public void deflate() {
this.data = null;
}


/**
* Sets the current currentLocation of the record. This should happen exactly-once
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,6 @@ public static void createHoodieProperties(FileSystem fs, Path metadataFolder, Pr
}
}


/**
* Read the table type from the table properties and if not found, return the default
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -267,7 +267,6 @@ public synchronized HoodieArchivedTimeline getArchivedTimeline() {
return archivedTimeline;
}


/**
* Helper method to initialize a dataset, with given basePath, tableType, name, archiveFolder
*/
Expand Down Expand Up @@ -410,7 +409,6 @@ public String getCommitActionType() {
}
}


/**
* Helper method to scan all hoodie-instant metafiles and construct HoodieInstant objects
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -294,7 +294,6 @@ private HoodieLogFormat.LogFormatVersion readVersion() throws IOException {
return new HoodieLogFormatVersion(inputStream.readInt());
}


private boolean readMagic() throws IOException {
try {
boolean hasMagic = hasNextMagic();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,6 @@ interface Reader extends Closeable, Iterator<HoodieLogBlock> {
public HoodieLogBlock prev() throws IOException;
}


/**
* Builder class to construct the default log format writer
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,6 @@ public HoodieDeleteBlock(HoodieKey[] keysToDelete, Map<HeaderMetadataType, Strin
this.keysToDelete = keysToDelete;
}


private HoodieDeleteBlock(Option<byte[]> content, FSDataInputStream inputStream, boolean readBlockLazily,
Option<HoodieLogBlockContentLocation> blockContentLocation, Map<HeaderMetadataType, String> header,
Map<HeaderMetadataType, String> footer) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -171,7 +171,6 @@ public HoodieTimeline getTimelineOfActions(Set<String> actions) {
(Function<HoodieInstant, Option<byte[]>> & Serializable) this::getInstantDetails);
}


/**
* Get only the cleaner action (inflight and completed) in the active timeline
*/
Expand Down Expand Up @@ -364,7 +363,6 @@ public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedI
return inflight;
}


private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option<byte[]> data) {
Preconditions.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp()));
Path commitFilePath = new Path(metaClient.getMetaPath(), toInstant.getFileName());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,6 @@ private void readObject(java.io.ObjectInputStream in) throws IOException, ClassN
in.defaultReadObject();
}


public static Path getArchiveLogPath(String archiveFolder) {
return new Path(archiveFolder, HOODIE_COMMIT_ARCHIVE_LOG_FILE);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.hudi.common.table.timeline.HoodieInstant;


@JsonIgnoreProperties(ignoreUnknown = true)
public class InstantDTO {

Expand Down
Loading