HUDI-70 : Making DeltaStreamer run in continuous mode with concurrent compaction
This commit is contained in:
committed by
Balaji Varadarajan
parent
3a210ef08e
commit
a0d7ab2384
@@ -85,7 +85,7 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
throws DatasetNotFoundException {
|
||||
log.info("Loading HoodieTableMetaClient from " + basePath);
|
||||
this.basePath = basePath;
|
||||
this.hadoopConf = new SerializableConfiguration(conf);
|
||||
this.hadoopConf = new SerializableConfiguration(new Configuration(conf));
|
||||
Path basePathDir = new Path(this.basePath);
|
||||
this.metaPath = basePath + File.separator + METAFOLDER_NAME;
|
||||
Path metaPathDir = new Path(this.metaPath);
|
||||
|
||||
@@ -74,7 +74,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
|
||||
super(config.isIncrementalTimelineSyncEnabled());
|
||||
this.config = config;
|
||||
this.schemaHelper = new RocksDBSchemaHelper(metaClient);
|
||||
this.rocksDB = new RocksDBDAO(metaClient.getBasePath(), config);
|
||||
this.rocksDB = new RocksDBDAO(metaClient.getBasePath(), config.getRocksdbBasePath());
|
||||
init(metaClient, visibleActiveTimeline);
|
||||
}
|
||||
|
||||
@@ -138,7 +138,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
|
||||
protected void resetViewState() {
|
||||
log.info("Deleting all rocksdb data associated with dataset filesystem view");
|
||||
rocksDB.close();
|
||||
rocksDB = new RocksDBDAO(metaClient.getBasePath(), config);
|
||||
rocksDB = new RocksDBDAO(metaClient.getBasePath(), config.getRocksdbBasePath());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -174,4 +174,12 @@ public class CompactionUtils {
|
||||
return Stream.empty();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return all pending compaction instant times
|
||||
* @return
|
||||
*/
|
||||
public static List<HoodieInstant> getPendingCompactionInstantTimes(HoodieTableMetaClient metaClient) {
|
||||
return metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,7 +20,6 @@ package com.uber.hoodie.common.util;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
@@ -57,7 +56,6 @@ public class RocksDBDAO {
|
||||
|
||||
protected static final transient Logger log = LogManager.getLogger(RocksDBDAO.class);
|
||||
|
||||
private final FileSystemViewStorageConfig config;
|
||||
private transient ConcurrentHashMap<String, ColumnFamilyHandle> managedHandlesMap;
|
||||
private transient ConcurrentHashMap<String, ColumnFamilyDescriptor> managedDescriptorMap;
|
||||
private transient RocksDB rocksDB;
|
||||
@@ -65,10 +63,9 @@ public class RocksDBDAO {
|
||||
private final String basePath;
|
||||
private final String rocksDBBasePath;
|
||||
|
||||
public RocksDBDAO(String basePath, FileSystemViewStorageConfig config) {
|
||||
public RocksDBDAO(String basePath, String rocksDBBasePath) {
|
||||
this.basePath = basePath;
|
||||
this.config = config;
|
||||
this.rocksDBBasePath = String.format("%s/%s/%s", config.getRocksdbBasePath(),
|
||||
this.rocksDBBasePath = String.format("%s/%s/%s", rocksDBBasePath,
|
||||
this.basePath.replace("/", "_"), UUID.randomUUID().toString());
|
||||
init();
|
||||
}
|
||||
@@ -95,6 +92,7 @@ public class RocksDBDAO {
|
||||
managedDescriptorMap = new ConcurrentHashMap<>();
|
||||
|
||||
// If already present, loads the existing column-family handles
|
||||
|
||||
final DBOptions dbOptions = new DBOptions().setCreateIfMissing(true).setCreateMissingColumnFamilies(true)
|
||||
.setWalDir(rocksDBBasePath).setStatsDumpPeriodSec(300).setStatistics(new Statistics());
|
||||
dbOptions.setLogger(new org.rocksdb.Logger(dbOptions) {
|
||||
@@ -183,6 +181,26 @@ public class RocksDBDAO {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to add put operation in batch
|
||||
*
|
||||
* @param batch Batch Handle
|
||||
* @param columnFamilyName Column Family
|
||||
* @param key Key
|
||||
* @param value Payload
|
||||
* @param <T> Type of payload
|
||||
*/
|
||||
public <K extends Serializable, T extends Serializable> void putInBatch(WriteBatch batch, String columnFamilyName,
|
||||
K key, T value) {
|
||||
try {
|
||||
byte[] keyBytes = SerializationUtils.serialize(key);
|
||||
byte[] payload = SerializationUtils.serialize(value);
|
||||
batch.put(managedHandlesMap.get(columnFamilyName), keyBytes, payload);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform single PUT on a column-family
|
||||
*
|
||||
@@ -200,6 +218,23 @@ public class RocksDBDAO {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform single PUT on a column-family
|
||||
*
|
||||
* @param columnFamilyName Column family name
|
||||
* @param key Key
|
||||
* @param value Payload
|
||||
* @param <T> Type of Payload
|
||||
*/
|
||||
public <K extends Serializable, T extends Serializable> void put(String columnFamilyName, K key, T value) {
|
||||
try {
|
||||
byte[] payload = SerializationUtils.serialize(value);
|
||||
getRocksDB().put(managedHandlesMap.get(columnFamilyName), SerializationUtils.serialize(key), payload);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to add delete operation in batch
|
||||
*
|
||||
@@ -215,6 +250,21 @@ public class RocksDBDAO {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to add delete operation in batch
|
||||
*
|
||||
* @param batch Batch Handle
|
||||
* @param columnFamilyName Column Family
|
||||
* @param key Key
|
||||
*/
|
||||
public <K extends Serializable> void deleteInBatch(WriteBatch batch, String columnFamilyName, K key) {
|
||||
try {
|
||||
batch.delete(managedHandlesMap.get(columnFamilyName), SerializationUtils.serialize(key));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform a single Delete operation
|
||||
*
|
||||
@@ -229,6 +279,20 @@ public class RocksDBDAO {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform a single Delete operation
|
||||
*
|
||||
* @param columnFamilyName Column Family name
|
||||
* @param key Key to be deleted
|
||||
*/
|
||||
public <K extends Serializable> void delete(String columnFamilyName, K key) {
|
||||
try {
|
||||
getRocksDB().delete(managedHandlesMap.get(columnFamilyName), SerializationUtils.serialize(key));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve a value for a given key in a column family
|
||||
*
|
||||
@@ -246,6 +310,23 @@ public class RocksDBDAO {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve a value for a given key in a column family
|
||||
*
|
||||
* @param columnFamilyName Column Family Name
|
||||
* @param key Key to be retrieved
|
||||
* @param <T> Type of object stored.
|
||||
*/
|
||||
public <K extends Serializable, T extends Serializable> T get(String columnFamilyName, K key) {
|
||||
Preconditions.checkArgument(!closed);
|
||||
try {
|
||||
byte[] val = getRocksDB().get(managedHandlesMap.get(columnFamilyName), SerializationUtils.serialize(key));
|
||||
return val == null ? null : SerializationUtils.deserialize(val);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform a prefix search and return stream of key-value pairs retrieved
|
||||
*
|
||||
|
||||
@@ -51,7 +51,7 @@ import org.apache.log4j.Logger;
|
||||
* without any rollover support. It uses the following : 1) An in-memory map that tracks the key-> latest ValueMetadata.
|
||||
* 2) Current position in the file NOTE : Only String.class type supported for Key
|
||||
*/
|
||||
public final class DiskBasedMap<T extends Serializable, R extends Serializable> implements Map<T, R> {
|
||||
public final class DiskBasedMap<T extends Serializable, R extends Serializable> implements Map<T, R>, Iterable<R> {
|
||||
|
||||
private static final Logger log = LogManager.getLogger(DiskBasedMap.class);
|
||||
// Stores the key and corresponding value's latest metadata spilled to disk
|
||||
@@ -149,6 +149,7 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
|
||||
/**
|
||||
* Custom iterator to iterate over values written to disk
|
||||
*/
|
||||
@Override
|
||||
public Iterator<R> iterator() {
|
||||
return new LazyFileIterable(filePath, valueMetadataMap).iterator();
|
||||
}
|
||||
|
||||
@@ -0,0 +1,125 @@
|
||||
/*
|
||||
* 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 com.uber.hoodie.common.util.collection;
|
||||
|
||||
import com.uber.hoodie.common.util.RocksDBDAO;
|
||||
import com.uber.hoodie.exception.HoodieNotSupportedException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public final class RocksDBBasedMap<K extends Serializable, R extends Serializable> implements Map<K, R> {
|
||||
|
||||
private static final String COL_FAMILY_NAME = "map_handle";
|
||||
|
||||
private final String rocksDbStoragePath;
|
||||
private RocksDBDAO rocksDBDAO;
|
||||
private final String columnFamilyName;
|
||||
|
||||
public RocksDBBasedMap(String rocksDbStoragePath) {
|
||||
this.rocksDbStoragePath = rocksDbStoragePath;
|
||||
this.columnFamilyName = COL_FAMILY_NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return (int)getRocksDBDAO().prefixSearch(columnFamilyName, "").count();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEmpty() {
|
||||
return size() == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsKey(Object key) {
|
||||
// Wont be able to store nulls as values
|
||||
return getRocksDBDAO().get(columnFamilyName, key.toString()) != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsValue(Object value) {
|
||||
throw new HoodieNotSupportedException("Not Supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public R get(Object key) {
|
||||
return getRocksDBDAO().get(columnFamilyName, (Serializable)key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public R put(K key, R value) {
|
||||
getRocksDBDAO().put(columnFamilyName, key, value);
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public R remove(Object key) {
|
||||
R val = getRocksDBDAO().get(columnFamilyName, key.toString());
|
||||
getRocksDBDAO().delete(columnFamilyName, key.toString());
|
||||
return val;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void putAll(Map<? extends K, ? extends R> m) {
|
||||
getRocksDBDAO().writeBatch(batch -> {
|
||||
m.entrySet().forEach(entry -> {
|
||||
getRocksDBDAO().putInBatch(batch, columnFamilyName, entry.getKey(), entry.getValue());
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
private RocksDBDAO getRocksDBDAO() {
|
||||
if (null == rocksDBDAO) {
|
||||
rocksDBDAO = new RocksDBDAO("default", rocksDbStoragePath);
|
||||
rocksDBDAO.addColumnFamily(columnFamilyName);
|
||||
}
|
||||
return rocksDBDAO;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
if (null != rocksDBDAO) {
|
||||
rocksDBDAO.close();
|
||||
}
|
||||
rocksDBDAO = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<K> keySet() {
|
||||
throw new HoodieNotSupportedException("Not Supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<R> values() {
|
||||
throw new HoodieNotSupportedException("Not Supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Entry<K, R>> entrySet() {
|
||||
throw new HoodieNotSupportedException("Not Supported");
|
||||
}
|
||||
|
||||
public Iterator<R> iterator() {
|
||||
return getRocksDBDAO().prefixSearch(columnFamilyName, "")
|
||||
.map(p -> (R)(p.getValue())).iterator();
|
||||
}
|
||||
}
|
||||
@@ -67,7 +67,7 @@ public class TestRocksDBManager {
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
dbManager = new RocksDBDAO("/dummy/path",
|
||||
FileSystemViewStorageConfig.newBuilder().build().newBuilder().build());
|
||||
FileSystemViewStorageConfig.newBuilder().build().newBuilder().build().getRocksdbBasePath());
|
||||
colFamilies.stream().forEach(family -> dbManager.dropColumnFamily(family));
|
||||
colFamilies.stream().forEach(family -> dbManager.addColumnFamily(family));
|
||||
|
||||
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* 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 com.uber.hoodie.common.util.collection;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.util.SchemaTestUtil;
|
||||
import com.uber.hoodie.common.util.SpillableMapTestUtils;
|
||||
import java.io.IOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestRocksDbBasedMap {
|
||||
|
||||
private static final String BASE_OUTPUT_PATH = "/tmp/";
|
||||
|
||||
@Test
|
||||
public void testSimple() throws IOException, URISyntaxException {
|
||||
RocksDBBasedMap records = new RocksDBBasedMap(BASE_OUTPUT_PATH);
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
((GenericRecord) iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
|
||||
|
||||
// make sure records have spilled to disk
|
||||
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
|
||||
List<HoodieRecord> oRecords = new ArrayList<>();
|
||||
while (itr.hasNext()) {
|
||||
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
|
||||
oRecords.add(rec);
|
||||
assert recordKeys.contains(rec.getRecordKey());
|
||||
}
|
||||
Assert.assertEquals(recordKeys.size(), oRecords.size());
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user