1
0

Removing compaction action type and associated compaction timeline operations, replace with commit action type

This commit is contained in:
Nishith Agarwal
2017-12-05 00:58:53 -08:00
committed by vinoth chandar
parent a1c0d0dbad
commit 44839b88c6
34 changed files with 265 additions and 450 deletions

View File

@@ -21,7 +21,6 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
@@ -35,17 +34,6 @@ import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.InvalidDatasetException;
import com.uber.hoodie.hive.util.SchemaUtil;
import java.io.IOException;
import java.sql.Connection;
import java.sql.DatabaseMetaData;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.commons.dbcp.BasicDataSource;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -63,6 +51,18 @@ import parquet.hadoop.ParquetFileReader;
import parquet.hadoop.metadata.ParquetMetadata;
import parquet.schema.MessageType;
import java.io.IOException;
import java.sql.Connection;
import java.sql.DatabaseMetaData;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
@SuppressWarnings("ConstantConditions")
public class HoodieHiveClient {
@@ -111,7 +111,7 @@ public class HoodieHiveClient {
e);
}
activeTimeline = metaClient.getActiveTimeline().getCommitsAndCompactionsTimeline()
activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants();
}
@@ -323,7 +323,7 @@ public class HoodieHiveClient {
// If this is MOR, depending on whether the latest commit is a delta commit or compaction commit
// Get a datafile written and get the schema from that file
Optional<HoodieInstant> lastCompactionCommit = metaClient.getActiveTimeline()
.getCompactionTimeline().filterCompletedInstants().lastInstant();
.getCommitTimeline().filterCompletedInstants().lastInstant();
LOG.info("Found the last compaction commit as " + lastCompactionCommit);
Optional<HoodieInstant> lastDeltaCommit;
@@ -379,7 +379,7 @@ public class HoodieHiveClient {
+ syncConfig.basePath));
// Read from the compacted file wrote
HoodieCompactionMetadata compactionMetadata = HoodieCompactionMetadata
HoodieCommitMetadata compactionMetadata = HoodieCommitMetadata
.fromBytes(activeTimeline.getInstantDetails(lastCompactionCommit).get());
String filePath = compactionMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values()
.stream().findAny()