HUDI-123 Rename code packages/constants to org.apache.hudi (#830)
- Rename com.uber.hoodie to org.apache.hudi - Flag to pass com.uber.hoodie Input formats for hoodie-sync - Works with HUDI demo. - Also tested for backwards compatibility with datasets built by com.uber.hoodie packages - Migration guide : https://cwiki.apache.org/confluence/display/HUDI/Migration+Guide+From+com.uber.hoodie+to+org.apache.hudi
This commit is contained in:
committed by
vinoth chandar
parent
722b6be04a
commit
a4f9d7575f
@@ -0,0 +1,45 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestBloomFilter {
|
||||
|
||||
@Test
|
||||
public void testAddKey() {
|
||||
BloomFilter filter = new BloomFilter(100, 0.0000001);
|
||||
filter.add("key1");
|
||||
assert (filter.mightContain("key1"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSerialize() throws IOException, ClassNotFoundException {
|
||||
BloomFilter filter = new BloomFilter(1000, 0.0000001);
|
||||
filter.add("key1");
|
||||
filter.add("key2");
|
||||
String filterStr = filter.serializeToString();
|
||||
|
||||
// Rebuild
|
||||
BloomFilter newFilter = new BloomFilter(filterStr);
|
||||
assert (newFilter.mightContain("key1"));
|
||||
assert (newFilter.mightContain("key2"));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,151 @@
|
||||
/*
|
||||
* 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.minicluster;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.io.Files;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* An HDFS minicluster service implementation.
|
||||
*/
|
||||
public class HdfsTestService {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(HdfsTestService.class);
|
||||
|
||||
/**
|
||||
* Configuration settings
|
||||
*/
|
||||
private Configuration hadoopConf;
|
||||
private String workDir;
|
||||
private String bindIP = "127.0.0.1";
|
||||
private int namenodeRpcPort = 8020;
|
||||
private int namenodeHttpPort = 50070;
|
||||
private int datanodePort = 50010;
|
||||
private int datanodeIpcPort = 50020;
|
||||
private int datanodeHttpPort = 50075;
|
||||
|
||||
/**
|
||||
* Embedded HDFS cluster
|
||||
*/
|
||||
private MiniDFSCluster miniDfsCluster;
|
||||
|
||||
public HdfsTestService() {
|
||||
workDir = Files.createTempDir().getAbsolutePath();
|
||||
}
|
||||
|
||||
public Configuration getHadoopConf() {
|
||||
return hadoopConf;
|
||||
}
|
||||
|
||||
public MiniDFSCluster start(boolean format) throws IOException {
|
||||
Preconditions.checkState(workDir != null, "The work dir must be set before starting cluster.");
|
||||
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
|
||||
|
||||
// If clean, then remove the work dir so we can start fresh.
|
||||
String localDFSLocation = getDFSLocation(workDir);
|
||||
if (format) {
|
||||
logger.info("Cleaning HDFS cluster data at: " + localDFSLocation + " and starting fresh.");
|
||||
File file = new File(localDFSLocation);
|
||||
FileUtils.deleteDirectory(file);
|
||||
}
|
||||
|
||||
// Configure and start the HDFS cluster
|
||||
// boolean format = shouldFormatDFSCluster(localDFSLocation, clean);
|
||||
hadoopConf = configureDFSCluster(hadoopConf, localDFSLocation, bindIP, namenodeRpcPort, namenodeHttpPort,
|
||||
datanodePort, datanodeIpcPort, datanodeHttpPort);
|
||||
miniDfsCluster = new MiniDFSCluster.Builder(hadoopConf).numDataNodes(1).format(format).checkDataNodeAddrConfig(true)
|
||||
.checkDataNodeHostConfig(true).build();
|
||||
logger.info("HDFS Minicluster service started.");
|
||||
return miniDfsCluster;
|
||||
}
|
||||
|
||||
public void stop() throws IOException {
|
||||
logger.info("HDFS Minicluster service being shut down.");
|
||||
miniDfsCluster.shutdown();
|
||||
miniDfsCluster = null;
|
||||
hadoopConf = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the location on the local FS where we store the HDFS data.
|
||||
*
|
||||
* @param baseFsLocation The base location on the local filesystem we have write access to create dirs.
|
||||
* @return The location for HDFS data.
|
||||
*/
|
||||
private static String getDFSLocation(String baseFsLocation) {
|
||||
return baseFsLocation + Path.SEPARATOR + "dfs";
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if we should format the DFS Cluster. We'll format if clean is true, or if the dfsFsLocation does not
|
||||
* exist.
|
||||
*
|
||||
* @param localDFSLocation The location on the local FS to hold the HDFS metadata and block data
|
||||
* @param clean Specifies if we want to start a clean cluster
|
||||
* @return Returns true if we should format a DFSCluster, otherwise false
|
||||
*/
|
||||
private static boolean shouldFormatDFSCluster(String localDFSLocation, boolean clean) {
|
||||
boolean format = true;
|
||||
File f = new File(localDFSLocation);
|
||||
if (f.exists() && f.isDirectory() && !clean) {
|
||||
format = false;
|
||||
}
|
||||
return format;
|
||||
}
|
||||
|
||||
/**
|
||||
* Configure the DFS Cluster before launching it.
|
||||
*
|
||||
* @param config The already created Hadoop configuration we'll further configure for HDFS
|
||||
* @param localDFSLocation The location on the local filesystem where cluster data is stored
|
||||
* @param bindIP An IP address we want to force the datanode and namenode to bind to.
|
||||
* @return The updated Configuration object.
|
||||
*/
|
||||
private static Configuration configureDFSCluster(Configuration config, String localDFSLocation, String bindIP,
|
||||
int namenodeRpcPort, int namenodeHttpPort, int datanodePort, int datanodeIpcPort, int datanodeHttpPort) {
|
||||
|
||||
logger.info("HDFS force binding to ip: " + bindIP);
|
||||
config.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + bindIP + ":" + namenodeRpcPort);
|
||||
config.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, bindIP + ":" + datanodePort);
|
||||
config.set(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, bindIP + ":" + datanodeIpcPort);
|
||||
config.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, bindIP + ":" + datanodeHttpPort);
|
||||
// When a datanode registers with the namenode, the Namenode do a hostname
|
||||
// check of the datanode which will fail on OpenShift due to reverse DNS
|
||||
// issues with the internal IP addresses. This config disables that check,
|
||||
// and will allow a datanode to connect regardless.
|
||||
config.setBoolean("dfs.namenode.datanode.registration.ip-hostname-check", false);
|
||||
config.set("hdfs.minidfs.basedir", localDFSLocation);
|
||||
// allow current user to impersonate others
|
||||
String user = System.getProperty("user.name");
|
||||
config.set("hadoop.proxyuser." + user + ".groups", "*");
|
||||
config.set("hadoop.proxyuser." + user + ".hosts", "*");
|
||||
return config;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* 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.minicluster;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.zookeeper.server.ZooKeeperServer;
|
||||
|
||||
public class MiniClusterUtil {
|
||||
|
||||
private static MiniDFSCluster dfsCluster;
|
||||
private static ZooKeeperServer zkServer;
|
||||
public static Configuration configuration;
|
||||
public static FileSystem fileSystem;
|
||||
|
||||
public static void setUp() throws IOException, InterruptedException {
|
||||
if (dfsCluster == null) {
|
||||
HdfsTestService service = new HdfsTestService();
|
||||
dfsCluster = service.start(true);
|
||||
configuration = service.getHadoopConf();
|
||||
}
|
||||
if (zkServer == null) {
|
||||
ZookeeperTestService zkService = new ZookeeperTestService(configuration);
|
||||
zkServer = zkService.start();
|
||||
}
|
||||
fileSystem = FileSystem.get(configuration);
|
||||
}
|
||||
|
||||
public static void shutdown() {
|
||||
if (dfsCluster != null) {
|
||||
dfsCluster.shutdown();
|
||||
}
|
||||
if (zkServer != null) {
|
||||
zkServer.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,237 @@
|
||||
/*
|
||||
* 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.minicluster;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.io.Files;
|
||||
import java.io.BufferedReader;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.io.OutputStream;
|
||||
import java.io.Reader;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.Socket;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.zookeeper.server.NIOServerCnxnFactory;
|
||||
import org.apache.zookeeper.server.ZooKeeperServer;
|
||||
import org.apache.zookeeper.server.persistence.FileTxnLog;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* A Zookeeper minicluster service implementation.
|
||||
* <p/>
|
||||
* This class was ripped from MiniZooKeeperCluster from the HBase tests. Changes made include:
|
||||
* <p/>
|
||||
* 1. It will now only launch 1 zookeeper server.
|
||||
* <p/>
|
||||
* 2. It will only attempt to bind to the port specified, and will fail if it can't.
|
||||
* <p/>
|
||||
* 3. The startup method now takes a bindAddress, which allows us to configure which IP the ZK server binds to. This was
|
||||
* not configurable in the original class.
|
||||
* <p/>
|
||||
* 4. The ZK cluster will re-use a data dir on the local filesystem if it already exists instead of blowing it away.
|
||||
*/
|
||||
public class ZookeeperTestService {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(ZookeeperTestService.class);
|
||||
|
||||
private static final int TICK_TIME = 2000;
|
||||
private static final int CONNECTION_TIMEOUT = 30000;
|
||||
|
||||
/**
|
||||
* Configuration settings
|
||||
*/
|
||||
private Configuration hadoopConf;
|
||||
private String workDir;
|
||||
private Integer clientPort = 2828;
|
||||
private String bindIP = "127.0.0.1";
|
||||
private Boolean clean = false;
|
||||
private int tickTime = 0;
|
||||
|
||||
/**
|
||||
* Embedded ZooKeeper cluster
|
||||
*/
|
||||
private NIOServerCnxnFactory standaloneServerFactory;
|
||||
private ZooKeeperServer zooKeeperServer;
|
||||
private boolean started = false;
|
||||
|
||||
public ZookeeperTestService(Configuration config) {
|
||||
this.workDir = Files.createTempDir().getAbsolutePath();
|
||||
this.hadoopConf = config;
|
||||
}
|
||||
|
||||
public Configuration getHadoopConf() {
|
||||
return hadoopConf;
|
||||
}
|
||||
|
||||
public ZooKeeperServer start() throws IOException, InterruptedException {
|
||||
Preconditions.checkState(workDir != null, "The localBaseFsLocation must be set before starting cluster.");
|
||||
|
||||
setupTestEnv();
|
||||
stop();
|
||||
|
||||
File dir = new File(workDir, "zookeeper").getAbsoluteFile();
|
||||
recreateDir(dir, clean);
|
||||
int tickTimeToUse;
|
||||
if (this.tickTime > 0) {
|
||||
tickTimeToUse = this.tickTime;
|
||||
} else {
|
||||
tickTimeToUse = TICK_TIME;
|
||||
}
|
||||
this.zooKeeperServer = new ZooKeeperServer(dir, dir, tickTimeToUse);
|
||||
standaloneServerFactory = new NIOServerCnxnFactory();
|
||||
|
||||
// NOTE: Changed from the original, where InetSocketAddress was
|
||||
// originally created to bind to the wildcard IP, we now configure it.
|
||||
logger.info("Zookeeper force binding to: " + this.bindIP);
|
||||
standaloneServerFactory.configure(new InetSocketAddress(bindIP, clientPort), 1000);
|
||||
|
||||
// Start up this ZK server
|
||||
standaloneServerFactory.startup(zooKeeperServer);
|
||||
|
||||
String serverHostname;
|
||||
if (bindIP.equals("0.0.0.0")) {
|
||||
serverHostname = "localhost";
|
||||
} else {
|
||||
serverHostname = bindIP;
|
||||
}
|
||||
if (!waitForServerUp(serverHostname, clientPort, CONNECTION_TIMEOUT)) {
|
||||
throw new IOException("Waiting for startup of standalone server");
|
||||
}
|
||||
|
||||
started = true;
|
||||
logger.info("Zookeeper Minicluster service started on client port: " + clientPort);
|
||||
return zooKeeperServer;
|
||||
}
|
||||
|
||||
public void stop() throws IOException {
|
||||
if (!started) {
|
||||
return;
|
||||
}
|
||||
|
||||
standaloneServerFactory.shutdown();
|
||||
if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
|
||||
throw new IOException("Waiting for shutdown of standalone server");
|
||||
}
|
||||
|
||||
// clear everything
|
||||
started = false;
|
||||
standaloneServerFactory = null;
|
||||
zooKeeperServer = null;
|
||||
|
||||
logger.info("Zookeeper Minicluster service shut down.");
|
||||
}
|
||||
|
||||
private void recreateDir(File dir, boolean clean) throws IOException {
|
||||
if (dir.exists() && clean) {
|
||||
FileUtil.fullyDelete(dir);
|
||||
} else if (dir.exists() && !clean) {
|
||||
// the directory's exist, and we don't want to clean, so exit
|
||||
return;
|
||||
}
|
||||
try {
|
||||
dir.mkdirs();
|
||||
} catch (SecurityException e) {
|
||||
throw new IOException("creating dir: " + dir, e);
|
||||
}
|
||||
}
|
||||
|
||||
// / XXX: From o.a.zk.t.ClientBase
|
||||
private static void setupTestEnv() {
|
||||
// during the tests we run with 100K prealloc in the logs.
|
||||
// on windows systems prealloc of 64M was seen to take ~15seconds
|
||||
// resulting in test failure (client timeout on first session).
|
||||
// set env and directly in order to handle static init/gc issues
|
||||
System.setProperty("zookeeper.preAllocSize", "100");
|
||||
FileTxnLog.setPreallocSize(100 * 1024);
|
||||
}
|
||||
|
||||
// XXX: From o.a.zk.t.ClientBase
|
||||
private static boolean waitForServerDown(int port, long timeout) {
|
||||
long start = System.currentTimeMillis();
|
||||
while (true) {
|
||||
try {
|
||||
Socket sock = new Socket("localhost", port);
|
||||
try {
|
||||
OutputStream outstream = sock.getOutputStream();
|
||||
outstream.write("stat".getBytes());
|
||||
outstream.flush();
|
||||
} finally {
|
||||
sock.close();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (System.currentTimeMillis() > start + timeout) {
|
||||
break;
|
||||
}
|
||||
try {
|
||||
Thread.sleep(250);
|
||||
} catch (InterruptedException e) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
// XXX: From o.a.zk.t.ClientBase
|
||||
private static boolean waitForServerUp(String hostname, int port, long timeout) {
|
||||
long start = System.currentTimeMillis();
|
||||
while (true) {
|
||||
try {
|
||||
Socket sock = new Socket(hostname, port);
|
||||
BufferedReader reader = null;
|
||||
try {
|
||||
OutputStream outstream = sock.getOutputStream();
|
||||
outstream.write("stat".getBytes());
|
||||
outstream.flush();
|
||||
|
||||
Reader isr = new InputStreamReader(sock.getInputStream());
|
||||
reader = new BufferedReader(isr);
|
||||
String line = reader.readLine();
|
||||
if (line != null && line.startsWith("Zookeeper version:")) {
|
||||
return true;
|
||||
}
|
||||
} finally {
|
||||
sock.close();
|
||||
if (reader != null) {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
// ignore as this is expected
|
||||
logger.info("server " + hostname + ":" + port + " not up " + e);
|
||||
}
|
||||
|
||||
if (System.currentTimeMillis() > start + timeout) {
|
||||
break;
|
||||
}
|
||||
try {
|
||||
Thread.sleep(250);
|
||||
} catch (InterruptedException e) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,61 @@
|
||||
/*
|
||||
* 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 java.io.IOException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
public class AvroBinaryTestPayload implements HoodieRecordPayload {
|
||||
|
||||
private final byte[] recordBytes;
|
||||
|
||||
public AvroBinaryTestPayload(Option<GenericRecord> record) {
|
||||
|
||||
try {
|
||||
if (record.isPresent()) {
|
||||
recordBytes = HoodieAvroUtils.avroToBytes(record.get());
|
||||
} else {
|
||||
recordBytes = new byte[0];
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("unable to convert payload to bytes");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieRecordPayload preCombine(HoodieRecordPayload another) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
|
||||
throws IOException {
|
||||
return getInsertValue(schema);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
|
||||
return Option.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,413 @@
|
||||
/*
|
||||
* 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.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import com.esotericsoftware.kryo.Kryo;
|
||||
import com.esotericsoftware.kryo.io.Input;
|
||||
import com.esotericsoftware.kryo.io.Output;
|
||||
import com.esotericsoftware.kryo.serializers.JavaSerializer;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Calendar;
|
||||
import java.util.Date;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
|
||||
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
public class HoodieTestUtils {
|
||||
|
||||
public static final String TEST_EXTENSION = ".test";
|
||||
public static final String RAW_TRIPS_TEST_NAME = "raw_trips";
|
||||
public static final String DEFAULT_WRITE_TOKEN = "1-0-1";
|
||||
public static final int DEFAULT_LOG_VERSION = 1;
|
||||
public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"};
|
||||
private static Random rand = new Random(46474747);
|
||||
|
||||
public static Configuration getDefaultHadoopConf() {
|
||||
return new Configuration();
|
||||
}
|
||||
|
||||
public static HoodieTableMetaClient init(String basePath)
|
||||
throws IOException {
|
||||
return init(basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
}
|
||||
|
||||
public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType) throws IOException {
|
||||
return initTableType(getDefaultHadoopConf(), basePath, tableType);
|
||||
}
|
||||
|
||||
public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath)
|
||||
throws IOException {
|
||||
return initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
}
|
||||
|
||||
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
|
||||
HoodieTableType tableType) throws IOException {
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME);
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, HoodieAvroPayload.class.getName());
|
||||
return HoodieTableMetaClient.initializePathAsHoodieDataset(hadoopConf, basePath, properties);
|
||||
}
|
||||
|
||||
public static String makeNewCommitTime() {
|
||||
return new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
}
|
||||
|
||||
public static final void createCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
for (String commitTime : commitTimes) {
|
||||
new File(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime))
|
||||
.createNewFile();
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createDeltaCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
for (String commitTime : commitTimes) {
|
||||
new File(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeDeltaFileName(commitTime))
|
||||
.createNewFile();
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createMetadataFolder(String basePath) throws IOException {
|
||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME).mkdirs();
|
||||
}
|
||||
|
||||
public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
for (String commitTime : commitTimes) {
|
||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeInflightCommitFileName(
|
||||
commitTime)).createNewFile();
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createInflightCleanFiles(String basePath, Configuration configuration, String... commitTimes)
|
||||
throws IOException {
|
||||
for (String commitTime : commitTimes) {
|
||||
Path commitFile = new Path((basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
|
||||
.makeInflightCleanerFileName(
|
||||
commitTime)));
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
FSDataOutputStream os = fs.create(commitFile, true);
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createInflightCleanFiles(String basePath, String... commitTimes) throws IOException {
|
||||
createInflightCleanFiles(basePath, HoodieTestUtils.getDefaultHadoopConf(), commitTimes);
|
||||
}
|
||||
|
||||
public static final String createNewDataFile(String basePath, String partitionPath, String commitTime)
|
||||
throws IOException {
|
||||
String fileID = UUID.randomUUID().toString();
|
||||
return createDataFile(basePath, partitionPath, commitTime, fileID);
|
||||
}
|
||||
|
||||
public static final String createNewMarkerFile(String basePath, String partitionPath, String commitTime)
|
||||
throws IOException {
|
||||
String fileID = UUID.randomUUID().toString();
|
||||
return createMarkerFile(basePath, partitionPath, commitTime, fileID);
|
||||
}
|
||||
|
||||
public static final String createDataFile(String basePath, String partitionPath, String commitTime, String fileID)
|
||||
throws IOException {
|
||||
String folderPath = basePath + "/" + partitionPath + "/";
|
||||
new File(folderPath).mkdirs();
|
||||
new File(folderPath + FSUtils.makeDataFileName(commitTime, DEFAULT_WRITE_TOKEN, fileID)).createNewFile();
|
||||
return fileID;
|
||||
}
|
||||
|
||||
public static final String createMarkerFile(String basePath, String partitionPath, String commitTime, String fileID)
|
||||
throws IOException {
|
||||
String folderPath = basePath + "/" + HoodieTableMetaClient.TEMPFOLDER_NAME + "/" + commitTime + "/"
|
||||
+ partitionPath + "/";
|
||||
new File(folderPath).mkdirs();
|
||||
File f = new File(folderPath + FSUtils.makeMarkerFile(commitTime, DEFAULT_WRITE_TOKEN, fileID));
|
||||
f.createNewFile();
|
||||
return f.getAbsolutePath();
|
||||
}
|
||||
|
||||
public static final String createNewLogFile(FileSystem fs, String basePath, String partitionPath, String commitTime,
|
||||
String fileID, Option<Integer> version) throws IOException {
|
||||
String folderPath = basePath + "/" + partitionPath + "/";
|
||||
boolean makeDir = fs.mkdirs(new Path(folderPath));
|
||||
if (!makeDir) {
|
||||
throw new IOException("cannot create directory for path " + folderPath);
|
||||
}
|
||||
boolean createFile = fs.createNewFile(new Path(
|
||||
folderPath + FSUtils
|
||||
.makeLogFileName(fileID, ".log", commitTime, version.orElse(DEFAULT_LOG_VERSION),
|
||||
HoodieLogFormat.UNKNOWN_WRITE_TOKEN)));
|
||||
if (!createFile) {
|
||||
throw new IOException(
|
||||
StringUtils.format("cannot create data file for commit %s and fileId %s", commitTime, fileID));
|
||||
}
|
||||
return fileID;
|
||||
}
|
||||
|
||||
public static final void createCompactionCommitFiles(FileSystem fs, String basePath, String... commitTimes)
|
||||
throws IOException {
|
||||
for (String commitTime : commitTimes) {
|
||||
boolean createFile = fs.createNewFile(new Path(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
|
||||
.makeCommitFileName(commitTime)));
|
||||
if (!createFile) {
|
||||
throw new IOException("cannot create commit file for commit " + commitTime);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createCompactionRequest(HoodieTableMetaClient metaClient, String instant,
|
||||
List<Pair<String, FileSlice>> fileSliceList) throws IOException {
|
||||
HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(fileSliceList, Option.empty(), Option.empty());
|
||||
HoodieInstant compactionInstant =
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instant);
|
||||
metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant,
|
||||
AvroUtils.serializeCompactionPlan(plan));
|
||||
}
|
||||
|
||||
public static final String getDataFilePath(String basePath, String partitionPath, String commitTime, String fileID) {
|
||||
return basePath + "/" + partitionPath + "/" + FSUtils
|
||||
.makeDataFileName(commitTime, DEFAULT_WRITE_TOKEN, fileID);
|
||||
}
|
||||
|
||||
public static final String getLogFilePath(String basePath, String partitionPath, String commitTime, String fileID,
|
||||
Option<Integer> version) {
|
||||
return basePath + "/" + partitionPath + "/" + FSUtils.makeLogFileName(fileID, ".log", commitTime,
|
||||
version.orElse(DEFAULT_LOG_VERSION), HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
|
||||
}
|
||||
|
||||
public static final String getCommitFilePath(String basePath, String commitTime) {
|
||||
return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION;
|
||||
}
|
||||
|
||||
public static final String getInflightCommitFilePath(String basePath, String commitTime) {
|
||||
return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime
|
||||
+ HoodieTimeline.INFLIGHT_COMMIT_EXTENSION;
|
||||
}
|
||||
|
||||
public static final String getRequestedCompactionFilePath(String basePath, String commitTime) {
|
||||
return basePath + "/" + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + "/" + commitTime
|
||||
+ HoodieTimeline.INFLIGHT_COMMIT_EXTENSION;
|
||||
}
|
||||
|
||||
public static final boolean doesDataFileExist(String basePath, String partitionPath, String commitTime,
|
||||
String fileID) {
|
||||
return new File(getDataFilePath(basePath, partitionPath, commitTime, fileID)).exists();
|
||||
}
|
||||
|
||||
public static final boolean doesLogFileExist(String basePath, String partitionPath, String commitTime, String fileID,
|
||||
Option<Integer> version) {
|
||||
return new File(getLogFilePath(basePath, partitionPath, commitTime, fileID, version)).exists();
|
||||
}
|
||||
|
||||
public static final boolean doesCommitExist(String basePath, String commitTime) {
|
||||
return new File(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION)
|
||||
.exists();
|
||||
}
|
||||
|
||||
public static final boolean doesInflightExist(String basePath, String commitTime) {
|
||||
return new File(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.INFLIGHT_EXTENSION)
|
||||
.exists();
|
||||
}
|
||||
|
||||
public static void createCleanFiles(String basePath, String commitTime, Configuration configuration)
|
||||
throws IOException {
|
||||
Path commitFile = new Path(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCleanerFileName(commitTime));
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
FSDataOutputStream os = fs.create(commitFile, true);
|
||||
try {
|
||||
HoodieCleanStat cleanStats = new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS,
|
||||
DEFAULT_PARTITION_PATHS[rand.nextInt(DEFAULT_PARTITION_PATHS.length)], new ArrayList<>(), new ArrayList<>(),
|
||||
new ArrayList<>(), commitTime);
|
||||
// Create the clean metadata
|
||||
HoodieCleanMetadata cleanMetadata = AvroUtils.convertCleanMetadata(commitTime, Option.of(0L),
|
||||
Arrays.asList(cleanStats));
|
||||
// Write empty clean metadata
|
||||
os.write(AvroUtils.serializeCleanMetadata(cleanMetadata).get());
|
||||
} finally {
|
||||
os.close();
|
||||
}
|
||||
}
|
||||
|
||||
public static void createCleanFiles(String basePath, String commitTime) throws IOException {
|
||||
createCleanFiles(basePath, commitTime, HoodieTestUtils.getDefaultHadoopConf());
|
||||
}
|
||||
|
||||
public static String makeTestFileName(String instant) {
|
||||
return instant + TEST_EXTENSION;
|
||||
}
|
||||
|
||||
public static String makeCommitFileName(String instant) {
|
||||
return instant + ".commit";
|
||||
}
|
||||
|
||||
public static void assertStreamEquals(String message, Stream<?> expected, Stream<?> actual) {
|
||||
Iterator<?> iter1 = expected.iterator();
|
||||
Iterator<?> iter2 = actual.iterator();
|
||||
while (iter1.hasNext() && iter2.hasNext()) {
|
||||
assertEquals(message, iter1.next(), iter2.next());
|
||||
}
|
||||
assert !iter1.hasNext() && !iter2.hasNext();
|
||||
}
|
||||
|
||||
public static <T extends Serializable> T serializeDeserialize(T object, Class<T> clazz)
|
||||
throws IOException, ClassNotFoundException {
|
||||
// Using Kyro as the default serializer in Spark Jobs
|
||||
Kryo kryo = new Kryo();
|
||||
kryo.register(HoodieTableMetaClient.class, new JavaSerializer());
|
||||
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
Output output = new Output(baos);
|
||||
kryo.writeObject(output, object);
|
||||
output.close();
|
||||
|
||||
Input input = new Input(new ByteArrayInputStream(baos.toByteArray()));
|
||||
T deseralizedObject = kryo.readObject(input, clazz);
|
||||
input.close();
|
||||
return deseralizedObject;
|
||||
}
|
||||
|
||||
public static void writeRecordsToLogFiles(FileSystem fs, String basePath, Schema schema,
|
||||
List<HoodieRecord> updatedRecords) {
|
||||
Map<HoodieRecordLocation, List<HoodieRecord>> groupedUpdated = updatedRecords.stream().collect(
|
||||
Collectors.groupingBy(HoodieRecord::getCurrentLocation));
|
||||
|
||||
groupedUpdated.entrySet().forEach(s -> {
|
||||
HoodieRecordLocation location = s.getKey();
|
||||
String partitionPath = s.getValue().get(0).getPartitionPath();
|
||||
|
||||
Writer logWriter;
|
||||
try {
|
||||
logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(basePath, partitionPath))
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(location.getFileId())
|
||||
.overBaseCommit(location.getInstantTime()).withFs(fs).build();
|
||||
|
||||
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, location.getInstantTime());
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
|
||||
logWriter.appendBlock(new HoodieAvroDataBlock(s.getValue().stream().map(r -> {
|
||||
try {
|
||||
GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get();
|
||||
HoodieAvroUtils.addHoodieKeyToRecord(val, r.getRecordKey(), r.getPartitionPath(), "");
|
||||
return (IndexedRecord) val;
|
||||
} catch (IOException e) {
|
||||
return null;
|
||||
}
|
||||
}).collect(Collectors.toList()), header));
|
||||
logWriter.close();
|
||||
} catch (Exception e) {
|
||||
fail(e.toString());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath) throws IOException {
|
||||
RemoteIterator<LocatedFileStatus> itr = fs.listFiles(new Path(basePath), true);
|
||||
List<FileStatus> returns = Lists.newArrayList();
|
||||
while (itr.hasNext()) {
|
||||
LocatedFileStatus status = itr.next();
|
||||
if (status.getPath().getName().contains(".parquet")) {
|
||||
returns.add(status);
|
||||
}
|
||||
}
|
||||
return returns.toArray(new FileStatus[returns.size()]);
|
||||
}
|
||||
|
||||
public static List<String> monotonicIncreasingCommitTimestamps(int numTimestamps, int startSecsDelta) {
|
||||
Calendar cal = Calendar.getInstance();
|
||||
cal.add(Calendar.SECOND, startSecsDelta);
|
||||
List<String> commits = new ArrayList<>();
|
||||
for (int i = 0; i < numTimestamps; i++) {
|
||||
commits.add(HoodieActiveTimeline.COMMIT_FORMATTER.format(cal.getTime()));
|
||||
cal.add(Calendar.SECOND, 1);
|
||||
}
|
||||
return commits;
|
||||
}
|
||||
|
||||
public static List<HoodieWriteStat> generateFakeHoodieWriteStat(int limit) {
|
||||
List<HoodieWriteStat> writeStatList = new ArrayList<>();
|
||||
for (int i = 0; i < limit; i++) {
|
||||
HoodieWriteStat writeStat = new HoodieWriteStat();
|
||||
writeStat.setFileId(UUID.randomUUID().toString());
|
||||
writeStat.setNumDeletes(0);
|
||||
writeStat.setNumUpdateWrites(100);
|
||||
writeStat.setNumWrites(100);
|
||||
writeStat.setPath("/some/fake/path" + i);
|
||||
writeStat.setPartitionPath("/some/fake/partition/path" + i);
|
||||
writeStat.setTotalLogFilesCompacted(100L);
|
||||
RuntimeStats runtimeStats = new RuntimeStats();
|
||||
runtimeStats.setTotalScanTime(100);
|
||||
runtimeStats.setTotalCreateTime(100);
|
||||
runtimeStats.setTotalUpsertTime(100);
|
||||
writeStat.setRuntimeStats(runtimeStats);
|
||||
writeStatList.add(writeStat);
|
||||
}
|
||||
return writeStatList;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,45 @@
|
||||
/*
|
||||
* 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 java.util.List;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestHoodieCommitMetadata {
|
||||
|
||||
@Test
|
||||
public void testPerfStatPresenceInHoodieMetadata() throws Exception {
|
||||
|
||||
List<HoodieWriteStat> fakeHoodieWriteStats = HoodieTestUtils.generateFakeHoodieWriteStat(100);
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
fakeHoodieWriteStats.stream().forEach(stat -> commitMetadata.addWriteStat(stat.getPartitionPath(), stat));
|
||||
Assert.assertTrue(commitMetadata.getTotalCreateTime() > 0);
|
||||
Assert.assertTrue(commitMetadata.getTotalUpsertTime() > 0);
|
||||
Assert.assertTrue(commitMetadata.getTotalScanTime() > 0);
|
||||
Assert.assertTrue(commitMetadata.getTotalLogFilesCompacted() > 0);
|
||||
|
||||
String serializedCommitMetadata = commitMetadata.toJsonString();
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(serializedCommitMetadata,
|
||||
HoodieCommitMetadata.class);
|
||||
// Make sure timing metrics are not written to instant file
|
||||
Assert.assertTrue(metadata.getTotalScanTime() == 0);
|
||||
Assert.assertTrue(metadata.getTotalLogFilesCompacted() > 0);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,57 @@
|
||||
/*
|
||||
* 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.assertEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.util.UUID;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestHoodieWriteStat {
|
||||
|
||||
@Test
|
||||
public void testSetPaths() {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
String basePathString = "/data/tables/some-hoodie-table";
|
||||
String partitionPathString = "2017/12/31";
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String writeToken = "1-0-1";
|
||||
|
||||
Path basePath = new Path(basePathString);
|
||||
Path partitionPath = new Path(basePath, partitionPathString);
|
||||
Path tempPath = new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME);
|
||||
|
||||
Path finalizeFilePath = new Path(partitionPath, FSUtils.makeDataFileName(commitTime, writeToken, fileName));
|
||||
HoodieWriteStat writeStat = new HoodieWriteStat();
|
||||
writeStat.setPath(basePath, finalizeFilePath);
|
||||
assertEquals(finalizeFilePath, new Path(basePath, writeStat.getPath()));
|
||||
|
||||
// test for null tempFilePath
|
||||
writeStat = new HoodieWriteStat();
|
||||
writeStat.setPath(basePath, finalizeFilePath);
|
||||
assertEquals(finalizeFilePath, new Path(basePath, writeStat.getPath()));
|
||||
assertNull(writeStat.getTempPath());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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.storage;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestStorageSchemes {
|
||||
|
||||
@Test
|
||||
public void testStorageSchemes() {
|
||||
assertTrue(StorageSchemes.isSchemeSupported("hdfs"));
|
||||
assertFalse(StorageSchemes.isSchemeSupported("s2"));
|
||||
assertFalse(StorageSchemes.isAppendSupported("s3a"));
|
||||
assertFalse(StorageSchemes.isAppendSupported("gs"));
|
||||
assertTrue(StorageSchemes.isAppendSupported("viewfs"));
|
||||
try {
|
||||
StorageSchemes.isAppendSupported("s2");
|
||||
fail("Should throw exception for unsupported schemes");
|
||||
} catch (IllegalArgumentException ignore) {
|
||||
// expected.
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,136 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import java.io.IOException;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class HoodieTableMetaClientTest {
|
||||
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private String basePath;
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder tmpFolder = new TemporaryFolder();
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath());
|
||||
basePath = metaClient.getBasePath();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkMetadata() {
|
||||
assertEquals("Table name should be raw_trips", HoodieTestUtils.RAW_TRIPS_TEST_NAME,
|
||||
metaClient.getTableConfig().getTableName());
|
||||
assertEquals("Basepath should be the one assigned", basePath, metaClient.getBasePath());
|
||||
assertEquals("Metapath should be ${basepath}/.hoodie", basePath + "/.hoodie", metaClient.getMetaPath());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkSerDe() throws IOException, ClassNotFoundException {
|
||||
// check if this object is serialized and de-serialized, we are able to read from the file system
|
||||
HoodieTableMetaClient deseralizedMetaClient = HoodieTestUtils
|
||||
.serializeDeserialize(metaClient, HoodieTableMetaClient.class);
|
||||
assertNotNull(deseralizedMetaClient);
|
||||
HoodieActiveTimeline commitTimeline = deseralizedMetaClient.getActiveTimeline();
|
||||
HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
commitTimeline.createInflight(instant);
|
||||
commitTimeline.saveAsComplete(instant, Option.of("test-detail".getBytes()));
|
||||
commitTimeline = commitTimeline.reload();
|
||||
HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant);
|
||||
assertEquals("Commit should be 1 and completed", completedInstant, commitTimeline.getInstants().findFirst().get());
|
||||
assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(),
|
||||
commitTimeline.getInstantDetails(completedInstant).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkCommitTimeline() throws IOException {
|
||||
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
|
||||
HoodieTimeline activeCommitTimeline = activeTimeline.getCommitTimeline();
|
||||
assertTrue("Should be empty commit timeline", activeCommitTimeline.empty());
|
||||
|
||||
HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
activeTimeline.createInflight(instant);
|
||||
activeTimeline.saveAsComplete(instant, Option.of("test-detail".getBytes()));
|
||||
|
||||
// Commit timeline should not auto-reload every time getActiveCommitTimeline(), it should be cached
|
||||
activeTimeline = metaClient.getActiveTimeline();
|
||||
activeCommitTimeline = activeTimeline.getCommitTimeline();
|
||||
assertTrue("Should be empty commit timeline", activeCommitTimeline.empty());
|
||||
|
||||
HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant);
|
||||
activeTimeline = activeTimeline.reload();
|
||||
activeCommitTimeline = activeTimeline.getCommitTimeline();
|
||||
assertFalse("Should be the 1 commit we made", activeCommitTimeline.empty());
|
||||
assertEquals("Commit should be 1", completedInstant, activeCommitTimeline.getInstants().findFirst().get());
|
||||
assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(),
|
||||
activeCommitTimeline.getInstantDetails(completedInstant).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkArchiveCommitTimeline() throws IOException {
|
||||
Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath());
|
||||
SequenceFile.Writer writer = SequenceFile
|
||||
.createWriter(metaClient.getHadoopConf(), SequenceFile.Writer.file(archiveLogPath),
|
||||
SequenceFile.Writer.keyClass(Text.class),
|
||||
SequenceFile.Writer.valueClass(Text.class));
|
||||
|
||||
writer.append(new Text("1"), new Text("data1"));
|
||||
writer.append(new Text("2"), new Text("data2"));
|
||||
writer.append(new Text("3"), new Text("data3"));
|
||||
|
||||
IOUtils.closeStream(writer);
|
||||
|
||||
HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline();
|
||||
|
||||
HoodieInstant instant1 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
HoodieInstant instant2 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2");
|
||||
HoodieInstant instant3 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
|
||||
|
||||
assertEquals(Lists.newArrayList(instant1, instant2, instant3),
|
||||
archivedTimeline.getInstants().collect(Collectors.toList()));
|
||||
|
||||
assertArrayEquals(new Text("data1").getBytes(), archivedTimeline.getInstantDetails(instant1).get());
|
||||
assertArrayEquals(new Text("data2").getBytes(), archivedTimeline.getInstantDetails(instant2).get());
|
||||
assertArrayEquals(new Text("data3").getBytes(), archivedTimeline.getInstantDetails(instant3).get());
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
@@ -0,0 +1,143 @@
|
||||
/*
|
||||
* 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.table.log;
|
||||
|
||||
import static org.apache.hudi.common.util.SchemaTestUtil.getSimpleSchema;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hudi.common.minicluster.MiniClusterUtil;
|
||||
import org.apache.hudi.common.model.HoodieArchivedLogFile;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
|
||||
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
import org.apache.hudi.common.util.SchemaTestUtil;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* This class is intentionally using a different way of setting up the MiniDFSCluster and not relying on
|
||||
* {@link MiniClusterUtil} to reproduce append() issue : https://issues.apache.org/jira/browse/HDFS-6325
|
||||
* Reference : https://issues.apache.org/jira/secure/attachment/12645053/HDFS-6325.patch
|
||||
*/
|
||||
public class HoodieLogFormatAppendFailureTest {
|
||||
|
||||
private static File baseDir;
|
||||
private static MiniDFSCluster cluster;
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpClass() throws IOException {
|
||||
// NOTE : The MiniClusterDFS leaves behind the directory under which the cluster was created
|
||||
baseDir = new File("/tmp/" + UUID.randomUUID().toString());
|
||||
FileUtil.fullyDelete(baseDir);
|
||||
// Append is not supported in LocalFileSystem. HDFS needs to be setup.
|
||||
Configuration conf = new Configuration();
|
||||
// lower heartbeat interval for fast recognition of DN
|
||||
conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
|
||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
|
||||
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
||||
conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 3000);
|
||||
cluster = new MiniDFSCluster.Builder(conf).checkExitOnShutdown(true).numDataNodes(4).build();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownClass() {
|
||||
cluster.shutdown(true);
|
||||
// Force clean up the directory under which the cluster was created
|
||||
FileUtil.fullyDelete(baseDir);
|
||||
}
|
||||
|
||||
@Test(timeout = 60000)
|
||||
public void testFailedToGetAppendStreamFromHDFSNameNode() throws IOException, URISyntaxException,
|
||||
InterruptedException, TimeoutException {
|
||||
|
||||
// Use some fs like LocalFileSystem, that does not support appends
|
||||
String uuid = UUID.randomUUID().toString();
|
||||
Path localPartitionPath = new Path("/tmp/");
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
Path testPath = new Path(localPartitionPath, uuid);
|
||||
fs.mkdirs(testPath);
|
||||
|
||||
// Some data & append.
|
||||
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 10);
|
||||
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 = HoodieLogFormat.newWriterBuilder().onParentPath(testPath)
|
||||
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits"
|
||||
+ ".archive").overBaseCommit("")
|
||||
.withFs(fs).build();
|
||||
|
||||
writer = writer.appendBlock(dataBlock);
|
||||
// get the current log file version to compare later
|
||||
int logFileVersion = writer.getLogFile().getLogVersion();
|
||||
Path logFilePath = writer.getLogFile().getPath();
|
||||
writer.close();
|
||||
|
||||
// Wait for 3 times replication of file
|
||||
DFSTestUtil.waitReplication(fs, logFilePath, (short) 3);
|
||||
// Shut down all DNs that have the last block location for the file
|
||||
LocatedBlocks lbs = cluster.getFileSystem().getClient().getNamenode()
|
||||
.getBlockLocations("/tmp/" + uuid + "/" + logFilePath.getName(), 0, Long.MAX_VALUE);
|
||||
List<DataNode> dnsOfCluster = cluster.getDataNodes();
|
||||
DatanodeInfo[] dnsWithLocations = lbs.getLastLocatedBlock().getLocations();
|
||||
for (DataNode dn : dnsOfCluster) {
|
||||
for (DatanodeInfo loc : dnsWithLocations) {
|
||||
if (dn.getDatanodeId().equals(loc)) {
|
||||
dn.shutdown();
|
||||
cluster.stopDataNode(dn.getDisplayName());
|
||||
DFSTestUtil.waitForDatanodeDeath(dn);
|
||||
}
|
||||
}
|
||||
}
|
||||
// Wait for the replication of this file to go down to 0
|
||||
DFSTestUtil.waitReplication(fs, logFilePath, (short) 0);
|
||||
|
||||
// Opening a new Writer right now will throw IOException. The code should handle this, rollover the logfile and
|
||||
// return a new writer with a bumped up logVersion
|
||||
writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath)
|
||||
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits" + ".archive")
|
||||
.overBaseCommit("")
|
||||
.withFs(fs).build();
|
||||
// The log version should be different for this new writer
|
||||
Assert.assertFalse(writer.getLogFile().getLogVersion() == logFileVersion);
|
||||
}
|
||||
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@@ -0,0 +1,124 @@
|
||||
/*
|
||||
* 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.table.string;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class HoodieActiveTimelineTest {
|
||||
|
||||
private HoodieActiveTimeline timeline;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
@Rule
|
||||
public final ExpectedException exception = ExpectedException.none();
|
||||
@Rule
|
||||
public TemporaryFolder tmpFolder = new TemporaryFolder();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
this.metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLoadingInstantsFromFiles() throws IOException {
|
||||
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "3");
|
||||
HoodieInstant instant3 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "5");
|
||||
HoodieInstant instant4 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "8");
|
||||
HoodieInstant instant1Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
HoodieInstant instant2Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
|
||||
HoodieInstant instant3Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "5");
|
||||
HoodieInstant instant4Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "8");
|
||||
|
||||
HoodieInstant instant5 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9");
|
||||
|
||||
timeline = new HoodieActiveTimeline(metaClient);
|
||||
timeline.saveAsComplete(instant1, Option.empty());
|
||||
timeline.saveAsComplete(instant2, Option.empty());
|
||||
timeline.saveAsComplete(instant3, Option.empty());
|
||||
timeline.saveAsComplete(instant4, Option.empty());
|
||||
timeline.createInflight(instant5);
|
||||
timeline = timeline.reload();
|
||||
|
||||
assertEquals("Total instants should be 5", 5, timeline.countInstants());
|
||||
HoodieTestUtils.assertStreamEquals("Check the instants stream",
|
||||
Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete, instant5),
|
||||
timeline.getInstants());
|
||||
HoodieTestUtils.assertStreamEquals("Check the instants stream",
|
||||
Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete, instant5),
|
||||
timeline.getCommitTimeline().getInstants());
|
||||
HoodieTestUtils.assertStreamEquals("Check the instants stream",
|
||||
Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete),
|
||||
timeline.getCommitTimeline().filterCompletedInstants().getInstants());
|
||||
HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream.of(instant5),
|
||||
timeline.getCommitTimeline().filterInflightsExcludingCompaction().getInstants());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimelineOperationsBasic() throws Exception {
|
||||
timeline = new HoodieActiveTimeline(metaClient);
|
||||
assertTrue(timeline.empty());
|
||||
assertEquals("", 0, timeline.countInstants());
|
||||
assertEquals("", Option.empty(), timeline.firstInstant());
|
||||
assertEquals("", Option.empty(), timeline.nthInstant(5));
|
||||
assertEquals("", Option.empty(), timeline.nthInstant(-1));
|
||||
assertEquals("", Option.empty(), timeline.lastInstant());
|
||||
assertFalse("", timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "01")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimelineOperations() throws Exception {
|
||||
timeline = new MockHoodieTimeline(Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"),
|
||||
Stream.of("21", "23"));
|
||||
HoodieTestUtils.assertStreamEquals("", Stream.of("05", "07", "09", "11"),
|
||||
timeline.getCommitTimeline().filterCompletedInstants().findInstantsInRange("04", "11").getInstants()
|
||||
.map(HoodieInstant::getTimestamp));
|
||||
HoodieTestUtils.assertStreamEquals("", Stream.of("09", "11"),
|
||||
timeline.getCommitTimeline().filterCompletedInstants().findInstantsAfter("07", 2).getInstants()
|
||||
.map(HoodieInstant::getTimestamp));
|
||||
assertFalse(timeline.empty());
|
||||
assertFalse(timeline.getCommitTimeline().filterInflightsExcludingCompaction().empty());
|
||||
assertEquals("", 12, timeline.countInstants());
|
||||
HoodieTimeline activeCommitTimeline = timeline.getCommitTimeline().filterCompletedInstants();
|
||||
assertEquals("", 10, activeCommitTimeline.countInstants());
|
||||
|
||||
assertEquals("", "01", activeCommitTimeline.firstInstant().get().getTimestamp());
|
||||
assertEquals("", "11", activeCommitTimeline.nthInstant(5).get().getTimestamp());
|
||||
assertEquals("", "19", activeCommitTimeline.lastInstant().get().getTimestamp());
|
||||
assertEquals("", "09", activeCommitTimeline.nthFromLastInstant(5).get().getTimestamp());
|
||||
assertTrue("", activeCommitTimeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "09")));
|
||||
assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
|
||||
assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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.table.string;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
|
||||
public class MockHoodieTimeline extends HoodieActiveTimeline {
|
||||
|
||||
public MockHoodieTimeline(Stream<String> completed, Stream<String> inflights) throws IOException {
|
||||
super();
|
||||
this.setInstants(Stream.concat(completed.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)),
|
||||
inflights.map(s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s)))
|
||||
.sorted(Comparator.comparing(new Function<HoodieInstant, String>() {
|
||||
@Override
|
||||
public String apply(HoodieInstant hoodieInstant) {
|
||||
return hoodieInstant.getFileName();
|
||||
}
|
||||
})).collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@@ -0,0 +1,791 @@
|
||||
/*
|
||||
* 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.table.view;
|
||||
|
||||
import static org.apache.hudi.common.table.HoodieTimeline.COMPACTION_ACTION;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterators;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroup;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class IncrementalFSViewSyncTest {
|
||||
|
||||
private static final transient Logger log = LogManager.getLogger(IncrementalFSViewSyncTest.class);
|
||||
|
||||
private static String TEST_WRITE_TOKEN = "1-0-1";
|
||||
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
protected String basePath;
|
||||
|
||||
private final List<String> partitions = Arrays.asList("2018/01/01", "2018/01/02",
|
||||
"2019/03/01");
|
||||
private final List<String> fileIdsPerPartition =
|
||||
IntStream.range(0, 10).mapToObj(x -> UUID.randomUUID().toString()).collect(Collectors.toList());
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder tmpFolder = new TemporaryFolder();
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
initializeMetaClient();
|
||||
refreshFsView();
|
||||
}
|
||||
|
||||
protected void initializeMetaClient() throws IOException {
|
||||
metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
|
||||
basePath = metaClient.getBasePath();
|
||||
partitions.forEach(p -> new File(basePath + "/" + p).mkdirs());
|
||||
}
|
||||
|
||||
protected void refreshFsView() throws IOException {
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
}
|
||||
|
||||
protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient) throws IOException {
|
||||
return getNewFileSystemView(metaClient, metaClient.getActiveTimeline().filterCompletedAndCompactionInstants());
|
||||
}
|
||||
|
||||
protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline timeline)
|
||||
throws IOException {
|
||||
return new HoodieTableFileSystemView(metaClient, timeline, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptyPartitionsAndTimeline() throws IOException {
|
||||
SyncableFileSystemView view = getNewFileSystemView(metaClient);
|
||||
Assert.assertFalse(view.getLastInstant().isPresent());
|
||||
partitions.forEach(p -> Assert.assertEquals(0, view.getLatestFileSlices(p).count()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAsyncCompaction() throws IOException {
|
||||
SyncableFileSystemView view = getNewFileSystemView(metaClient);
|
||||
view.sync();
|
||||
|
||||
// Run 3 ingestion on MOR table (3 delta commits)
|
||||
Map<String, List<String>> instantsToFiles =
|
||||
testMultipleWriteSteps(view, Arrays.asList("11", "12", "13"), true, "11");
|
||||
|
||||
// Schedule Compaction
|
||||
scheduleCompaction(view, "14");
|
||||
|
||||
// Restore pending compaction
|
||||
unscheduleCompaction(view, "14", "13", "11");
|
||||
|
||||
// Add one more delta instant
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("15"), true, "11"));
|
||||
|
||||
// Schedule Compaction again
|
||||
scheduleCompaction(view, "16");
|
||||
|
||||
// Run Compaction - This will be the second file-slice
|
||||
testMultipleWriteSteps(view, Arrays.asList("16"), false, "16", 2);
|
||||
|
||||
// Run 2 more ingest
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("17", "18"), true, "16", 2));
|
||||
|
||||
// Schedule Compaction again
|
||||
scheduleCompaction(view, "19");
|
||||
|
||||
// Run one more ingestion after pending compaction. THis will be 3rd slice
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("20"), true, "19", 3));
|
||||
|
||||
// Clean first slice
|
||||
testCleans(view, Arrays.asList("21"),
|
||||
new ImmutableMap.Builder<String, List<String>>().put("11", Arrays.asList("12", "13", "15")).build(),
|
||||
instantsToFiles,
|
||||
Arrays.asList("11"));
|
||||
|
||||
// Add one more ingestion instant. This should be 2nd slice now
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("22"), true, "19", 2));
|
||||
|
||||
// Restore last ingestion
|
||||
testRestore(view, Arrays.asList("23"), true, new HashMap<>(), Arrays.asList("22"), "24", false);
|
||||
|
||||
// Run one more ingestion. THis is still 2nd slice
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("24"), true, "19", 2));
|
||||
|
||||
// Finish Compaction
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("19"), false, "19", 2,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "24"))));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIngestion() throws IOException {
|
||||
SyncableFileSystemView view = getNewFileSystemView(metaClient);
|
||||
|
||||
// Add an empty ingestion
|
||||
String firstEmptyInstantTs = "11";
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
|
||||
view.sync();
|
||||
Assert.assertTrue(view.getLastInstant().isPresent());
|
||||
Assert.assertEquals("11", view.getLastInstant().get().getTimestamp());
|
||||
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
|
||||
Assert.assertEquals(HoodieTimeline.COMMIT_ACTION, view.getLastInstant().get().getAction());
|
||||
partitions.forEach(p -> Assert.assertEquals(0, view.getLatestFileSlices(p).count()));
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
for (String partition : partitions) {
|
||||
newView.getAllFileGroups(partition).count();
|
||||
}
|
||||
|
||||
areViewsConsistent(view, newView, 0L);
|
||||
|
||||
// Add 3 non-empty ingestions to COW table
|
||||
Map<String, List<String>> instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("12", "13", "14"));
|
||||
|
||||
// restore instants in reverse order till we rollback all
|
||||
testRestore(view, Arrays.asList("15", "16", "17"), false, instantsToFiles,
|
||||
Arrays.asList("14", "13", "12"), "17", true);
|
||||
|
||||
// Add 5 non-empty ingestions back-to-back
|
||||
instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("18", "19", "20"));
|
||||
|
||||
// Clean instants.
|
||||
testCleans(view, Arrays.asList("21", "22"), instantsToFiles, Arrays.asList("18", "19"));
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests FS View incremental syncing behavior when multiple instants gets committed
|
||||
*/
|
||||
@Test
|
||||
public void testMultipleTransitions() throws IOException {
|
||||
|
||||
SyncableFileSystemView view1 = getNewFileSystemView(metaClient);
|
||||
view1.sync();
|
||||
Map<String, List<String>> instantsToFiles = null;
|
||||
|
||||
/**
|
||||
* Case where incremental syncing is catching up on more than one ingestion at a time
|
||||
*/
|
||||
// Run 1 ingestion on MOR table (1 delta commits). View1 is now sync up to this point
|
||||
instantsToFiles =
|
||||
testMultipleWriteSteps(view1, Arrays.asList("11"), true, "11");
|
||||
|
||||
SyncableFileSystemView view2 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
|
||||
// Run 2 more ingestion on MOR table. View1 is not yet synced but View2 is
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("12", "13"), true, "11"));
|
||||
|
||||
// Now Sync view1 and add 1 more ingestion. Check if view1 is able to catchup correctly
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view1, Arrays.asList("14"), true, "11"));
|
||||
|
||||
view2.sync();
|
||||
SyncableFileSystemView view3 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
partitions.stream().forEach(p -> view3.getLatestFileSlices(p).count());
|
||||
view3.sync();
|
||||
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size());
|
||||
|
||||
/**
|
||||
* Case where a compaction is scheduled and then unscheduled
|
||||
*/
|
||||
scheduleCompaction(view2, "15");
|
||||
unscheduleCompaction(view2, "15", "14", "11");
|
||||
view1.sync();
|
||||
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size());
|
||||
SyncableFileSystemView view4 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
partitions.stream().forEach(p -> view4.getLatestFileSlices(p).count());
|
||||
view4.sync();
|
||||
|
||||
/**
|
||||
* Case where a compaction is scheduled, 2 ingestion happens and then a compaction happens
|
||||
*/
|
||||
scheduleCompaction(view2, "16");
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("17", "18"), true, "16", 2));
|
||||
// Compaction
|
||||
testMultipleWriteSteps(view2, Arrays.asList("16"), false, "16", 2,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "18")));
|
||||
view1.sync();
|
||||
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size() * 2);
|
||||
SyncableFileSystemView view5 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
partitions.stream().forEach(p -> view5.getLatestFileSlices(p).count());
|
||||
view5.sync();
|
||||
|
||||
/**
|
||||
* Case where a clean happened and then rounds of ingestion and compaction happened
|
||||
*/
|
||||
testCleans(view2, Arrays.asList("19"),
|
||||
new ImmutableMap.Builder<String, List<String>>().put("11", Arrays.asList("12", "13", "14")).build(),
|
||||
instantsToFiles,
|
||||
Arrays.asList("11"));
|
||||
scheduleCompaction(view2, "20");
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("21", "22"), true, "20", 2));
|
||||
// Compaction
|
||||
testMultipleWriteSteps(view2, Arrays.asList("20"), false, "20", 2,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "22")));
|
||||
//Run one more round of ingestion
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("23", "24"), true, "20", 2));
|
||||
view1.sync();
|
||||
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size() * 2);
|
||||
SyncableFileSystemView view6 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
partitions.stream().forEach(p -> view5.getLatestFileSlices(p).count());
|
||||
view6.sync();
|
||||
|
||||
/**
|
||||
* Case where multiple restores and ingestions happened
|
||||
*/
|
||||
testRestore(view2, Arrays.asList("25"), true, new HashMap<>(), Arrays.asList("24"), "29", true);
|
||||
testRestore(view2, Arrays.asList("26"), true, new HashMap<>(), Arrays.asList("23"), "29", false);
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("27"), true, "20", 2));
|
||||
scheduleCompaction(view2, "28");
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("29"), true, "28", 3));
|
||||
// Compaction
|
||||
testMultipleWriteSteps(view2, Arrays.asList("28"), false, "28", 3,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "29")));
|
||||
|
||||
Arrays.asList(view1, view2, view3, view4, view5, view6).stream().forEach(v -> {
|
||||
v.sync();
|
||||
areViewsConsistent(v, view1, partitions.size() * fileIdsPerPartition.size() * 3);
|
||||
});
|
||||
}
|
||||
|
||||
/*
|
||||
********************************************************************************************************
|
||||
* HELPER METHODS
|
||||
*********************************************************************************************************
|
||||
*/
|
||||
|
||||
|
||||
/**
|
||||
* Helper to run one or more rounds of cleaning, incrementally syncing the view and then validate
|
||||
*/
|
||||
private void testCleans(SyncableFileSystemView view, List<String> newCleanerInstants,
|
||||
Map<String, List<String>> instantsToFiles, List<String> cleanedInstants) {
|
||||
Map<String, List<String>> deltaInstantMap = cleanedInstants.stream().map(e -> Pair.of(e, new ArrayList()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
testCleans(view, newCleanerInstants, deltaInstantMap, instantsToFiles, cleanedInstants);
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulates one of more cleaning, incrementally sync the view and validate the view
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param newCleanerInstants Cleaner Instants
|
||||
* @param deltaInstantMap File-Slice Base Instants to Delta Instants
|
||||
* @param instantsToFiles List of files associated with each instant
|
||||
* @param cleanedInstants List of cleaned instants
|
||||
*/
|
||||
private void testCleans(SyncableFileSystemView view, List<String> newCleanerInstants,
|
||||
Map<String, List<String>> deltaInstantMap,
|
||||
Map<String, List<String>> instantsToFiles, List<String> cleanedInstants) {
|
||||
Assert.assertEquals(newCleanerInstants.size(), cleanedInstants.size());
|
||||
long initialFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).findAny().getAsLong();
|
||||
long exp = initialFileSlices;
|
||||
log.info("Initial File Slices :" + exp);
|
||||
for (int idx = 0; idx < newCleanerInstants.size(); idx++) {
|
||||
String instant = cleanedInstants.get(idx);
|
||||
try {
|
||||
List<String> filesToDelete = new ArrayList<>(instantsToFiles.get(instant));
|
||||
deltaInstantMap.get(instant).stream().forEach(n -> filesToDelete.addAll(instantsToFiles.get(n)));
|
||||
|
||||
performClean(view, instant, filesToDelete, newCleanerInstants.get(idx));
|
||||
|
||||
exp -= fileIdsPerPartition.size();
|
||||
final long expTotalFileSlicesPerPartition = exp;
|
||||
view.sync();
|
||||
Assert.assertTrue(view.getLastInstant().isPresent());
|
||||
Assert.assertEquals(newCleanerInstants.get(idx), view.getLastInstant().get().getTimestamp());
|
||||
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
|
||||
Assert.assertEquals(HoodieTimeline.CLEAN_ACTION, view.getLastInstant().get().getAction());
|
||||
partitions.forEach(p -> {
|
||||
log.info("PARTTITION : " + p);
|
||||
log.info("\tFileSlices :" + view.getAllFileSlices(p).collect(Collectors.toList()));
|
||||
});
|
||||
|
||||
partitions.forEach(p -> Assert.assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count()));
|
||||
partitions.forEach(p -> Assert.assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
for (String partition : partitions) {
|
||||
newView.getAllFileGroups(partition).count();
|
||||
}
|
||||
areViewsConsistent(view, newView, expTotalFileSlicesPerPartition * partitions.size());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulates one of more restores, incrementally sync the view and validate the view
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param newRestoreInstants Restore Instants
|
||||
* @param isDeltaCommit is Delta Commit ?
|
||||
* @param instantsToFiles List of files associated with each instant
|
||||
* @param rolledBackInstants List of rolled-back instants
|
||||
* @param emptyRestoreInstant Restore instant at which dataset becomes empty
|
||||
*/
|
||||
private void testRestore(SyncableFileSystemView view, List<String> newRestoreInstants, boolean isDeltaCommit,
|
||||
Map<String, List<String>> instantsToFiles, List<String> rolledBackInstants,
|
||||
String emptyRestoreInstant, boolean isRestore)
|
||||
throws IOException {
|
||||
Assert.assertEquals(newRestoreInstants.size(), rolledBackInstants.size());
|
||||
long initialFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).findAny().getAsLong();
|
||||
IntStream.range(0, newRestoreInstants.size()).forEach(idx -> {
|
||||
String instant = rolledBackInstants.get(idx);
|
||||
try {
|
||||
performRestore(view, instant, instantsToFiles.get(instant), newRestoreInstants.get(idx), isRestore);
|
||||
final long expTotalFileSlicesPerPartition = isDeltaCommit ? initialFileSlices :
|
||||
initialFileSlices - ((idx + 1) * fileIdsPerPartition.size());
|
||||
view.sync();
|
||||
Assert.assertTrue(view.getLastInstant().isPresent());
|
||||
log.info("Last Instant is :" + view.getLastInstant().get());
|
||||
if (isRestore) {
|
||||
Assert.assertEquals(newRestoreInstants.get(idx), view.getLastInstant().get().getTimestamp());
|
||||
Assert.assertEquals(isRestore ? HoodieTimeline.RESTORE_ACTION : HoodieTimeline.ROLLBACK_ACTION,
|
||||
view.getLastInstant().get().getAction());
|
||||
}
|
||||
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
|
||||
|
||||
if (HoodieTimeline.compareTimestamps(newRestoreInstants.get(idx), emptyRestoreInstant,
|
||||
HoodieTimeline.GREATER_OR_EQUAL)) {
|
||||
partitions.forEach(p -> Assert.assertEquals(0, view.getLatestFileSlices(p).count()));
|
||||
} else {
|
||||
partitions.forEach(p -> Assert.assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count()));
|
||||
}
|
||||
partitions.forEach(p -> Assert.assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
for (String partition : partitions) {
|
||||
newView.getAllFileGroups(partition).count();
|
||||
}
|
||||
areViewsConsistent(view, newView, expTotalFileSlicesPerPartition * partitions.size());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulate a Cleaner operation cleaning up an instant
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instant Instant to be cleaner
|
||||
* @param files List of files to be deleted
|
||||
* @param cleanInstant Cleaner Instant
|
||||
*/
|
||||
private void performClean(SyncableFileSystemView view, String instant, List<String> files, String cleanInstant)
|
||||
throws IOException {
|
||||
Map<String, List<String>> partititonToFiles = deleteFiles(files);
|
||||
List<HoodieCleanStat> cleanStats = partititonToFiles.entrySet().stream().map(e -> {
|
||||
return new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_COMMITS,
|
||||
e.getKey(), e.getValue(), e.getValue(), new ArrayList<>(),
|
||||
Integer.toString(Integer.parseInt(instant) + 1));
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
HoodieCleanMetadata cleanMetadata = AvroUtils.convertCleanMetadata(cleanInstant, Option.empty(), cleanStats);
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, cleanInstant),
|
||||
AvroUtils.serializeCleanMetadata(cleanMetadata));
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulate Restore of an instant in timeline and fsview
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instant Instant to be rolled-back
|
||||
* @param files List of files to be deleted as part of rollback
|
||||
* @param rollbackInstant Restore Instant
|
||||
*/
|
||||
private void performRestore(SyncableFileSystemView view, String instant, List<String> files, String rollbackInstant,
|
||||
boolean isRestore) throws IOException {
|
||||
Map<String, List<String>> partititonToFiles = deleteFiles(files);
|
||||
List<HoodieRollbackStat> rollbackStats = partititonToFiles.entrySet().stream().map(e -> {
|
||||
return new HoodieRollbackStat(e.getKey(), e.getValue(), new ArrayList<>(), new HashMap<>());
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
List<String> rollbacks = new ArrayList<>();
|
||||
rollbacks.add(instant);
|
||||
|
||||
HoodieRollbackMetadata rollbackMetadata = AvroUtils
|
||||
.convertRollbackMetadata(rollbackInstant, Option.empty(), rollbacks, rollbackStats);
|
||||
if (isRestore) {
|
||||
HoodieRestoreMetadata metadata = new HoodieRestoreMetadata();
|
||||
|
||||
List<HoodieRollbackMetadata> rollbackM = new ArrayList<>();
|
||||
rollbackM.add(rollbackMetadata);
|
||||
metadata.setHoodieRestoreMetadata(
|
||||
new ImmutableMap.Builder().put(rollbackInstant, rollbackM).build());
|
||||
List<String> rollbackInstants = new ArrayList<>();
|
||||
rollbackInstants.add(rollbackInstant);
|
||||
metadata.setInstantsToRollback(rollbackInstants);
|
||||
metadata.setStartRestoreTime(rollbackInstant);
|
||||
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, rollbackInstant),
|
||||
AvroUtils.serializeRestoreMetadata(metadata));
|
||||
} else {
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant),
|
||||
AvroUtils.serializeRollbackMetadata(rollbackMetadata));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility to delete a list of files and group the deleted files by partitions
|
||||
*
|
||||
* @param files List of files to be deleted
|
||||
*/
|
||||
private Map<String, List<String>> deleteFiles(List<String> files) {
|
||||
|
||||
if (null == files) {
|
||||
return new HashMap<>();
|
||||
}
|
||||
|
||||
Map<String, List<String>> partititonToFiles = new HashMap<>();
|
||||
partitions.forEach(p -> partititonToFiles.put(p, new ArrayList<>()));
|
||||
|
||||
for (String f : files) {
|
||||
String fullPath = String.format("%s/%s", metaClient.getBasePath(), f);
|
||||
new File(fullPath).delete();
|
||||
String partition = partitions.stream().filter(p -> f.startsWith(p)).findAny().get();
|
||||
partititonToFiles.get(partition).add(fullPath);
|
||||
}
|
||||
return partititonToFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Schedule a pending compaction and validate
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instantTime COmpaction Instant Time
|
||||
*/
|
||||
private void scheduleCompaction(SyncableFileSystemView view, String instantTime) throws IOException {
|
||||
List<Pair<String, FileSlice>> slices = partitions.stream().flatMap(p -> view.getLatestFileSlices(p)
|
||||
.map(s -> Pair.of(p, s))).collect(Collectors.toList());
|
||||
|
||||
long initialExpTotalFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).sum();
|
||||
|
||||
HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(slices, Option.empty(), Option.empty());
|
||||
HoodieInstant compactionInstant =
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime);
|
||||
metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant,
|
||||
AvroUtils.serializeCompactionPlan(plan));
|
||||
|
||||
view.sync();
|
||||
partitions.stream().forEach(p -> {
|
||||
view.getLatestFileSlices(p).forEach(fs -> {
|
||||
Assert.assertEquals(instantTime, fs.getBaseInstantTime());
|
||||
Assert.assertEquals(p, fs.getPartitionPath());
|
||||
Assert.assertFalse(fs.getDataFile().isPresent());
|
||||
});
|
||||
view.getLatestMergedFileSlicesBeforeOrOn(p, instantTime).forEach(fs -> {
|
||||
Assert.assertTrue(HoodieTimeline.compareTimestamps(instantTime, fs.getBaseInstantTime(),
|
||||
HoodieTimeline.GREATER));
|
||||
Assert.assertEquals(p, fs.getPartitionPath());
|
||||
});
|
||||
});
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
partitions.forEach(p -> newView.getLatestFileSlices(p).count());
|
||||
areViewsConsistent(view, newView, initialExpTotalFileSlices + partitions.size() * fileIdsPerPartition.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Unschedule a compaction instant and validate incremental fs view
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param compactionInstantTime Compaction Instant to be removed
|
||||
* @param newLastInstant New Last instant
|
||||
* @param newBaseInstant New Base instant of last file-slice
|
||||
*/
|
||||
private void unscheduleCompaction(SyncableFileSystemView view, String compactionInstantTime, String newLastInstant,
|
||||
String newBaseInstant) throws IOException {
|
||||
HoodieInstant instant = new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstantTime);
|
||||
boolean deleted = metaClient.getFs().delete(new Path(metaClient.getMetaPath(), instant.getFileName()), false);
|
||||
Preconditions.checkArgument(deleted, "Unable to delete compaction instant.");
|
||||
|
||||
view.sync();
|
||||
Assert.assertEquals(newLastInstant, view.getLastInstant().get().getTimestamp());
|
||||
partitions.stream().forEach(p -> {
|
||||
view.getLatestFileSlices(p).forEach(fs -> {
|
||||
Assert.assertEquals(newBaseInstant, fs.getBaseInstantTime());
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instants Ingestion/Commit INstants
|
||||
* @param deltaCommit Delta COmmit ?
|
||||
* @param baseInstantForDeltaCommit Base Instant to be used in case of delta-commit
|
||||
* @return List of new file created
|
||||
*/
|
||||
private Map<String, List<String>> testMultipleWriteSteps(SyncableFileSystemView view, List<String> instants,
|
||||
boolean deltaCommit, String baseInstantForDeltaCommit) throws IOException {
|
||||
return testMultipleWriteSteps(view, instants, deltaCommit, baseInstantForDeltaCommit, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instants Ingestion/Commit INstants
|
||||
* @param deltaCommit Delta COmmit ?
|
||||
* @param baseInstantForDeltaCommit Base Instant to be used in case of delta-commit
|
||||
* @param begin initial file-slice offset
|
||||
* @return List of new file created
|
||||
*/
|
||||
private Map<String, List<String>> testMultipleWriteSteps(SyncableFileSystemView view, List<String> instants,
|
||||
boolean deltaCommit, String baseInstantForDeltaCommit, int begin)
|
||||
throws IOException {
|
||||
return testMultipleWriteSteps(view, instants, deltaCommit, baseInstantForDeltaCommit, begin,
|
||||
instants.stream().map(i -> new HoodieInstant(State.COMPLETED,
|
||||
deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, i))
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instants Ingestion/Commit INstants
|
||||
* @return List of new file created
|
||||
*/
|
||||
private Map<String, List<String>> testMultipleWriteSteps(SyncableFileSystemView view, List<String> instants)
|
||||
throws IOException {
|
||||
return testMultipleWriteSteps(view, instants, false, null, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instants Ingestion/Commit INstants
|
||||
* @param deltaCommit Delta COmmit ?
|
||||
* @param baseInstantForDeltaCommit Base Instant to be used in case of delta-commit
|
||||
* @param begin initial file-slice offset
|
||||
* @param lastInstants List of Last Instants at each time we ingest/compact
|
||||
* @return List of new file created
|
||||
*/
|
||||
private Map<String, List<String>> testMultipleWriteSteps(SyncableFileSystemView view, List<String> instants,
|
||||
boolean deltaCommit, String baseInstantForDeltaCommit, int begin, List<HoodieInstant> lastInstants)
|
||||
throws IOException {
|
||||
Map<String, List<String>> instantToFiles = new HashMap<>();
|
||||
|
||||
int multiple = begin;
|
||||
for (int idx = 0; idx < instants.size(); idx++) {
|
||||
String instant = instants.get(idx);
|
||||
log.info("Adding instant=" + instant);
|
||||
HoodieInstant lastInstant = lastInstants.get(idx);
|
||||
// Add a non-empty ingestion to COW table
|
||||
List<String> filePaths = addInstant(metaClient, instant, deltaCommit,
|
||||
deltaCommit ? baseInstantForDeltaCommit : instant);
|
||||
view.sync();
|
||||
Assert.assertTrue(view.getLastInstant().isPresent());
|
||||
Assert.assertEquals(lastInstant.getTimestamp(), view.getLastInstant().get().getTimestamp());
|
||||
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
|
||||
Assert.assertEquals("Expected Last=" + lastInstant + ", Found Instants="
|
||||
+ view.getTimeline().getInstants().collect(Collectors.toList()),
|
||||
lastInstant.getAction(), view.getLastInstant().get().getAction());
|
||||
partitions.forEach(p -> Assert.assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count()));
|
||||
final long expTotalFileSlicesPerPartition = fileIdsPerPartition.size() * multiple;
|
||||
partitions.forEach(p -> Assert.assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
|
||||
if (deltaCommit) {
|
||||
partitions.forEach(p -> {
|
||||
view.getLatestFileSlices(p).forEach(f -> {
|
||||
Assert.assertEquals(baseInstantForDeltaCommit, f.getBaseInstantTime());
|
||||
});
|
||||
});
|
||||
} else {
|
||||
partitions.forEach(p -> {
|
||||
view.getLatestDataFiles(p).forEach(f -> {
|
||||
Assert.assertEquals(instant, f.getCommitTime());
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
for (String partition : partitions) {
|
||||
newView.getAllFileGroups(partition).count();
|
||||
}
|
||||
areViewsConsistent(view, newView, fileIdsPerPartition.size() * partitions.size() * multiple);
|
||||
instantToFiles.put(instant, filePaths);
|
||||
if (!deltaCommit) {
|
||||
multiple++;
|
||||
}
|
||||
}
|
||||
return instantToFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check for equality of views
|
||||
*
|
||||
* @param view1 View1
|
||||
* @param view2 View2
|
||||
*/
|
||||
private void areViewsConsistent(SyncableFileSystemView view1, SyncableFileSystemView view2,
|
||||
long expectedTotalFileSlices) {
|
||||
//Timeline check
|
||||
HoodieTimeline timeline1 = view1.getTimeline();
|
||||
HoodieTimeline timeline2 = view2.getTimeline();
|
||||
Assert.assertEquals(view1.getLastInstant(), view2.getLastInstant());
|
||||
Iterators.elementsEqual(timeline1.getInstants().iterator(), timeline2.getInstants().iterator());
|
||||
|
||||
//View Checks
|
||||
Map<HoodieFileGroupId, HoodieFileGroup> fileGroupsMap1 =
|
||||
partitions.stream().flatMap(p -> view1.getAllFileGroups(p))
|
||||
.collect(Collectors.toMap(fg -> fg.getFileGroupId(), fg -> fg));
|
||||
Map<HoodieFileGroupId, HoodieFileGroup> fileGroupsMap2 =
|
||||
partitions.stream().flatMap(p -> view2.getAllFileGroups(p))
|
||||
.collect(Collectors.toMap(fg -> fg.getFileGroupId(), fg -> fg));
|
||||
Assert.assertEquals(fileGroupsMap1.keySet(), fileGroupsMap2.keySet());
|
||||
long gotSlicesCount =
|
||||
fileGroupsMap1.keySet().stream().map(k -> Pair.of(fileGroupsMap1.get(k), fileGroupsMap2.get(k)))
|
||||
.mapToLong(e -> {
|
||||
HoodieFileGroup fg1 = e.getKey();
|
||||
HoodieFileGroup fg2 = e.getValue();
|
||||
Assert.assertEquals(fg1.getFileGroupId(), fg2.getFileGroupId());
|
||||
List<FileSlice> slices1 = fg1.getAllRawFileSlices().collect(Collectors.toList());
|
||||
List<FileSlice> slices2 = fg2.getAllRawFileSlices().collect(Collectors.toList());
|
||||
Assert.assertEquals(slices1.size(), slices2.size());
|
||||
IntStream.range(0, slices1.size()).mapToObj(idx -> Pair.of(slices1.get(idx), slices2.get(idx)))
|
||||
.forEach(e2 -> {
|
||||
FileSlice slice1 = e2.getKey();
|
||||
FileSlice slice2 = e2.getValue();
|
||||
Assert.assertEquals(slice1.getBaseInstantTime(), slice2.getBaseInstantTime());
|
||||
Assert.assertEquals(slice1.getFileId(), slice2.getFileId());
|
||||
Assert.assertEquals(slice1.getDataFile().isPresent(), slice2.getDataFile().isPresent());
|
||||
if (slice1.getDataFile().isPresent()) {
|
||||
HoodieDataFile df1 = slice1.getDataFile().get();
|
||||
HoodieDataFile df2 = slice2.getDataFile().get();
|
||||
Assert.assertEquals(df1.getCommitTime(), df2.getCommitTime());
|
||||
Assert.assertEquals(df1.getFileId(), df2.getFileId());
|
||||
Assert.assertEquals(df1.getFileName(), df2.getFileName());
|
||||
Assert.assertEquals(Path.getPathWithoutSchemeAndAuthority(new Path(df1.getPath())),
|
||||
Path.getPathWithoutSchemeAndAuthority(new Path(df2.getPath())));
|
||||
}
|
||||
List<Path> logPaths1 = slice1.getLogFiles()
|
||||
.map(lf -> Path.getPathWithoutSchemeAndAuthority(lf.getPath())).collect(Collectors.toList());
|
||||
List<Path> logPaths2 = slice2.getLogFiles()
|
||||
.map(lf -> Path.getPathWithoutSchemeAndAuthority(lf.getPath())).collect(Collectors.toList());
|
||||
Assert.assertEquals(logPaths1, logPaths2);
|
||||
});
|
||||
return slices1.size();
|
||||
}).sum();
|
||||
Assert.assertEquals(expectedTotalFileSlices, gotSlicesCount);
|
||||
|
||||
// Pending Compaction Operations Check
|
||||
Set<Pair<String, CompactionOperation>> ops1 = view1.getPendingCompactionOperations().collect(Collectors.toSet());
|
||||
Set<Pair<String, CompactionOperation>> ops2 = view2.getPendingCompactionOperations().collect(Collectors.toSet());
|
||||
Assert.assertEquals(ops1, ops2);
|
||||
}
|
||||
|
||||
private List<String> addInstant(HoodieTableMetaClient metaClient, String instant, boolean deltaCommit,
|
||||
String baseInstant)
|
||||
throws IOException {
|
||||
List<Pair<String, HoodieWriteStat>> writeStats = partitions.stream().flatMap(p -> {
|
||||
return fileIdsPerPartition.stream().map(f -> {
|
||||
try {
|
||||
File file = new File(basePath + "/" + p + "/"
|
||||
+ (deltaCommit ? FSUtils.makeLogFileName(f, ".log", baseInstant,
|
||||
Integer.parseInt(instant), TEST_WRITE_TOKEN) : FSUtils.makeDataFileName(instant, TEST_WRITE_TOKEN, f)));
|
||||
file.createNewFile();
|
||||
HoodieWriteStat w = new HoodieWriteStat();
|
||||
w.setFileId(f);
|
||||
w.setPath(String.format("%s/%s", p, file.getName()));
|
||||
return Pair.of(p, w);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
writeStats.forEach(e -> metadata.addWriteStat(e.getKey(), e.getValue()));
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION,
|
||||
instant),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
// Delete pending compaction if present
|
||||
metaClient.getFs().delete(new Path(metaClient.getMetaPath(),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instant).getFileName()));
|
||||
return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,35 @@
|
||||
/*
|
||||
* 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.table.view;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.SyncableFileSystemView;
|
||||
|
||||
public class RocksDBBasedIncrementalFSViewSyncTest extends IncrementalFSViewSyncTest {
|
||||
|
||||
@Override
|
||||
protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline timeline)
|
||||
throws IOException {
|
||||
return new RocksDbBasedFileSystemView(metaClient, timeline,
|
||||
FileSystemViewStorageConfig.newBuilder().withRocksDBPath(tmpFolder.newFolder().getAbsolutePath())
|
||||
.withIncrementalTimelineSync(true).build());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,32 @@
|
||||
/*
|
||||
* 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.table.view;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.SyncableFileSystemView;
|
||||
|
||||
public class RocksDbBasedFileSystemViewTest extends HoodieTableFileSystemViewTest {
|
||||
|
||||
protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) throws IOException {
|
||||
return new RocksDbBasedFileSystemView(metaClient, timeline,
|
||||
FileSystemViewStorageConfig.newBuilder().withRocksDBPath(tmpFolder.newFolder().getAbsolutePath())
|
||||
.build());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,31 @@
|
||||
/*
|
||||
* 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.table.view;
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.SyncableFileSystemView;
|
||||
|
||||
public class SpillableMapBasedFileSystemViewTest extends HoodieTableFileSystemViewTest {
|
||||
|
||||
protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) {
|
||||
return new SpillableMapBasedFileSystemView(metaClient, timeline,
|
||||
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.SPILLABLE_DISK)
|
||||
.withMaxMemoryForView(0L).build()); // pure disk base View
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,32 @@
|
||||
/*
|
||||
* 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.table.view;
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.SyncableFileSystemView;
|
||||
|
||||
public class SpillableMapBasedIncrementalFSViewSyncTest extends IncrementalFSViewSyncTest {
|
||||
|
||||
@Override
|
||||
protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline timeline) {
|
||||
return new SpillableMapBasedFileSystemView(metaClient, timeline,
|
||||
FileSystemViewStorageConfig.newBuilder().withMaxMemoryForView(0L).withIncrementalTimelineSync(true).build());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,212 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import static org.apache.hudi.common.model.HoodieTestUtils.DEFAULT_PARTITION_PATHS;
|
||||
import static org.apache.hudi.common.table.HoodieTimeline.COMPACTION_ACTION;
|
||||
import static org.apache.hudi.common.table.HoodieTimeline.DELTA_COMMIT_ACTION;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.junit.Assert;
|
||||
|
||||
public class CompactionTestUtils {
|
||||
|
||||
private static String TEST_WRITE_TOKEN = "1-0-1";
|
||||
|
||||
public static Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> setupAndValidateCompactionOperations(
|
||||
HoodieTableMetaClient metaClient, boolean inflight,
|
||||
int numEntriesInPlan1, int numEntriesInPlan2,
|
||||
int numEntriesInPlan3, int numEntriesInPlan4) throws IOException {
|
||||
HoodieCompactionPlan plan1 = createCompactionPlan(metaClient, "000", "001", numEntriesInPlan1, true, true);
|
||||
HoodieCompactionPlan plan2 = createCompactionPlan(metaClient, "002", "003", numEntriesInPlan2, false, true);
|
||||
HoodieCompactionPlan plan3 = createCompactionPlan(metaClient, "004", "005", numEntriesInPlan3, true, false);
|
||||
HoodieCompactionPlan plan4 = createCompactionPlan(metaClient, "006", "007", numEntriesInPlan4, false, false);
|
||||
|
||||
if (inflight) {
|
||||
scheduleInflightCompaction(metaClient, "001", plan1);
|
||||
scheduleInflightCompaction(metaClient, "003", plan2);
|
||||
scheduleInflightCompaction(metaClient, "005", plan3);
|
||||
scheduleInflightCompaction(metaClient, "007", plan4);
|
||||
} else {
|
||||
scheduleCompaction(metaClient, "001", plan1);
|
||||
scheduleCompaction(metaClient, "003", plan2);
|
||||
scheduleCompaction(metaClient, "005", plan3);
|
||||
scheduleCompaction(metaClient, "007", plan4);
|
||||
}
|
||||
|
||||
createDeltaCommit(metaClient, "000");
|
||||
createDeltaCommit(metaClient, "002");
|
||||
createDeltaCommit(metaClient, "004");
|
||||
createDeltaCommit(metaClient, "006");
|
||||
|
||||
Map<String, String> baseInstantsToCompaction =
|
||||
new ImmutableMap.Builder<String, String>().put("000", "001").put("002", "003")
|
||||
.put("004", "005").put("006", "007").build();
|
||||
List<Integer> expectedNumEntries =
|
||||
Arrays.asList(numEntriesInPlan1, numEntriesInPlan2, numEntriesInPlan3, numEntriesInPlan4);
|
||||
List<HoodieCompactionPlan> plans = new ImmutableList.Builder<HoodieCompactionPlan>()
|
||||
.add(plan1, plan2, plan3, plan4).build();
|
||||
IntStream.range(0, 4).boxed().forEach(idx -> {
|
||||
if (expectedNumEntries.get(idx) > 0) {
|
||||
Assert.assertEquals("check if plan " + idx + " has exp entries",
|
||||
expectedNumEntries.get(idx).longValue(), plans.get(idx).getOperations().size());
|
||||
} else {
|
||||
Assert.assertNull("Plan " + idx + " has null ops", plans.get(idx).getOperations());
|
||||
}
|
||||
});
|
||||
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath(), true);
|
||||
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> pendingCompactionMap =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient);
|
||||
|
||||
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> expPendingCompactionMap =
|
||||
generateExpectedCompactionOperations(Arrays.asList(plan1, plan2, plan3, plan4), baseInstantsToCompaction);
|
||||
|
||||
// Ensure Compaction operations are fine.
|
||||
Assert.assertEquals(expPendingCompactionMap, pendingCompactionMap);
|
||||
return expPendingCompactionMap;
|
||||
}
|
||||
|
||||
public static Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> generateExpectedCompactionOperations(
|
||||
List<HoodieCompactionPlan> plans, Map<String, String> baseInstantsToCompaction) {
|
||||
return plans.stream()
|
||||
.flatMap(plan -> {
|
||||
if (plan.getOperations() != null) {
|
||||
return plan.getOperations().stream().map(op -> Pair.of(
|
||||
new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()),
|
||||
Pair.of(baseInstantsToCompaction.get(op.getBaseInstantTime()), op)));
|
||||
}
|
||||
return Stream.empty();
|
||||
}).collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
}
|
||||
|
||||
public static void scheduleCompaction(HoodieTableMetaClient metaClient,
|
||||
String instantTime, HoodieCompactionPlan compactionPlan) throws IOException {
|
||||
metaClient.getActiveTimeline().saveToCompactionRequested(
|
||||
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime),
|
||||
AvroUtils.serializeCompactionPlan(compactionPlan));
|
||||
}
|
||||
|
||||
public static void createDeltaCommit(HoodieTableMetaClient metaClient, String instantTime) throws IOException {
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(State.INFLIGHT, DELTA_COMMIT_ACTION, instantTime), Option.empty());
|
||||
}
|
||||
|
||||
public static void scheduleInflightCompaction(HoodieTableMetaClient metaClient, String instantTime,
|
||||
HoodieCompactionPlan compactionPlan) throws IOException {
|
||||
scheduleCompaction(metaClient, instantTime, compactionPlan);
|
||||
metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(
|
||||
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime));
|
||||
}
|
||||
|
||||
public static HoodieCompactionPlan createCompactionPlan(HoodieTableMetaClient metaClient, String instantId,
|
||||
String compactionInstantId, int numFileIds, boolean createDataFile,
|
||||
boolean deltaCommitsAfterCompactionRequests) {
|
||||
List<HoodieCompactionOperation> ops = IntStream.range(0, numFileIds).boxed().map(idx -> {
|
||||
try {
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
if (createDataFile) {
|
||||
HoodieTestUtils.createDataFile(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId);
|
||||
}
|
||||
HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
|
||||
instantId, fileId, Option.of(1));
|
||||
HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
|
||||
instantId, fileId, Option.of(2));
|
||||
FileSlice slice = new FileSlice(DEFAULT_PARTITION_PATHS[0], instantId, fileId);
|
||||
if (createDataFile) {
|
||||
slice.setDataFile(new TestHoodieDataFile(metaClient.getBasePath() + "/" + DEFAULT_PARTITION_PATHS[0]
|
||||
+ "/" + FSUtils.makeDataFileName(instantId, TEST_WRITE_TOKEN, fileId)));
|
||||
}
|
||||
String logFilePath1 = HoodieTestUtils
|
||||
.getLogFilePath(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId,
|
||||
Option.of(1));
|
||||
String logFilePath2 = HoodieTestUtils
|
||||
.getLogFilePath(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId,
|
||||
Option.of(2));
|
||||
slice.addLogFile(new HoodieLogFile(new Path(logFilePath1)));
|
||||
slice.addLogFile(new HoodieLogFile(new Path(logFilePath2)));
|
||||
HoodieCompactionOperation op =
|
||||
CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], slice, Option.empty());
|
||||
if (deltaCommitsAfterCompactionRequests) {
|
||||
HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
|
||||
compactionInstantId, fileId, Option.of(1));
|
||||
HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
|
||||
compactionInstantId, fileId, Option.of(2));
|
||||
}
|
||||
return op;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}).collect(Collectors.toList());
|
||||
return new HoodieCompactionPlan(ops.isEmpty() ? null : ops, new HashMap<>());
|
||||
}
|
||||
|
||||
public static class TestHoodieDataFile extends HoodieDataFile {
|
||||
|
||||
private final String path;
|
||||
|
||||
public TestHoodieDataFile(String path) {
|
||||
super("/tmp/ce481ee7-9e53-4a2e-9992-f9e295fa79c0_11_20180918020003.parquet");
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPath() {
|
||||
return path;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getFileId() {
|
||||
return UUID.randomUUID().toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getCommitTime() {
|
||||
return "100";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFileSize() {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,185 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.nio.file.FileSystem;
|
||||
import java.nio.file.FileSystemNotFoundException;
|
||||
import java.nio.file.FileSystems;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.avro.io.DecoderFactory;
|
||||
import org.apache.hudi.avro.MercifulJsonConverter;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
public class SchemaTestUtil {
|
||||
|
||||
public static Schema getSimpleSchema() throws IOException {
|
||||
return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/simple-test.avsc"));
|
||||
}
|
||||
|
||||
public static List<IndexedRecord> generateTestRecords(int from, int limit) throws IOException, URISyntaxException {
|
||||
return toRecords(getSimpleSchema(), getSimpleSchema(), from, limit);
|
||||
}
|
||||
|
||||
private static List<IndexedRecord> toRecords(Schema writerSchema, Schema readerSchema, int from, int limit)
|
||||
throws IOException, URISyntaxException {
|
||||
GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema);
|
||||
// Required to register the necessary JAR:// file system
|
||||
URI resource = SchemaTestUtil.class.getClass().getResource("/sample.data").toURI();
|
||||
Path dataPath;
|
||||
if (resource.toString().contains("!")) {
|
||||
dataPath = uriToPath(resource);
|
||||
} else {
|
||||
dataPath = Paths.get(SchemaTestUtil.class.getClass().getResource("/sample.data").toURI());
|
||||
}
|
||||
|
||||
try (Stream<String> stream = Files.lines(dataPath)) {
|
||||
return stream.skip(from).limit(limit).map(s -> {
|
||||
try {
|
||||
return reader.read(null, DecoderFactory.get().jsonDecoder(writerSchema, s));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not read data from simple_data.json", e);
|
||||
}
|
||||
}).collect(Collectors.toList());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not read data from simple_data.json", e);
|
||||
}
|
||||
}
|
||||
|
||||
static Path uriToPath(URI uri) throws IOException {
|
||||
final Map<String, String> env = new HashMap<>();
|
||||
final String[] array = uri.toString().split("!");
|
||||
FileSystem fs;
|
||||
try {
|
||||
fs = FileSystems.getFileSystem(URI.create(array[0]));
|
||||
} catch (FileSystemNotFoundException e) {
|
||||
fs = FileSystems.newFileSystem(URI.create(array[0]), env);
|
||||
}
|
||||
return fs.getPath(array[1]);
|
||||
}
|
||||
|
||||
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());
|
||||
|
||||
}
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
private static HoodieRecord convertToHoodieRecords(IndexedRecord iRecord, String key, String partitionPath) {
|
||||
return new HoodieRecord<>(new HoodieKey(key, partitionPath),
|
||||
new HoodieAvroPayload(Option.of((GenericRecord) iRecord)));
|
||||
}
|
||||
|
||||
public static List<IndexedRecord> updateHoodieTestRecords(List<String> oldRecordKeys, List<IndexedRecord> newRecords,
|
||||
String commitTime) throws IOException, URISyntaxException {
|
||||
|
||||
return newRecords.stream()
|
||||
.map(p -> {
|
||||
((GenericRecord) p).put(HoodieRecord.RECORD_KEY_METADATA_FIELD, oldRecordKeys.remove(0));
|
||||
((GenericRecord) p).put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00");
|
||||
((GenericRecord) p).put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime);
|
||||
return p;
|
||||
}).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(Option.of((GenericRecord) r)))).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public static List<HoodieRecord> updateHoodieTestRecordsWithoutHoodieMetadata(List<HoodieRecord> oldRecords,
|
||||
Schema schema,
|
||||
String fieldNameToUpdate, String newValue)
|
||||
throws IOException, URISyntaxException {
|
||||
return oldRecords
|
||||
.stream()
|
||||
.map(r -> {
|
||||
try {
|
||||
GenericRecord rec = (GenericRecord) r.getData().getInsertValue(schema).get();
|
||||
rec.put(fieldNameToUpdate, newValue);
|
||||
return new HoodieRecord<>(r.getKey(),
|
||||
new HoodieAvroPayload(Option.of(rec)));
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("unable to get data from hoodie record", io);
|
||||
}
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public static Schema getEvolvedSchema() throws IOException {
|
||||
return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avsc"));
|
||||
}
|
||||
|
||||
public static List<IndexedRecord> generateEvolvedTestRecords(int from, int limit)
|
||||
throws IOException, URISyntaxException {
|
||||
return toRecords(getSimpleSchema(), getEvolvedSchema(), from, limit);
|
||||
}
|
||||
|
||||
public static Schema getComplexEvolvedSchema() throws IOException {
|
||||
return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/complex-test-evolved.avsc"));
|
||||
}
|
||||
|
||||
public static GenericRecord generateAvroRecordFromJson(Schema schema, int recordNumber, String commitTime,
|
||||
String fileId) throws IOException {
|
||||
TestRecord record = new TestRecord(commitTime, recordNumber, fileId);
|
||||
MercifulJsonConverter converter = new MercifulJsonConverter(schema);
|
||||
return converter.convert(record.toJsonString());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,53 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
|
||||
public class SpillableMapTestUtils {
|
||||
|
||||
public static final String DUMMY_COMMIT_TIME = "DUMMY_COMMIT_TIME";
|
||||
public static final String DUMMY_FILE_ID = "DUMMY_FILE_ID";
|
||||
|
||||
public static List<String> upsertRecords(List<IndexedRecord> iRecords,
|
||||
Map<String, HoodieRecord<? extends HoodieRecordPayload>> records) {
|
||||
List<String> recordKeys = new ArrayList<>();
|
||||
iRecords
|
||||
.stream()
|
||||
.forEach(r -> {
|
||||
String key = ((GenericRecord) r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String partitionPath = ((GenericRecord) r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
|
||||
recordKeys.add(key);
|
||||
HoodieRecord record = new HoodieRecord<>(new HoodieKey(key, partitionPath),
|
||||
new HoodieAvroPayload(Option.of((GenericRecord) r)));
|
||||
record.setCurrentLocation(new HoodieRecordLocation("DUMMY_COMMIT_TIME", "DUMMY_FILE_ID"));
|
||||
records.put(key, record);
|
||||
});
|
||||
return recordKeys;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,225 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import static org.apache.hudi.common.model.HoodieTestUtils.DEFAULT_PARTITION_PATHS;
|
||||
import static org.apache.hudi.common.model.HoodieTestUtils.getDefaultHadoopConf;
|
||||
import static org.apache.hudi.common.util.CompactionTestUtils.createCompactionPlan;
|
||||
import static org.apache.hudi.common.util.CompactionTestUtils.scheduleCompaction;
|
||||
import static org.apache.hudi.common.util.CompactionTestUtils.setupAndValidateCompactionOperations;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.CompactionTestUtils.TestHoodieDataFile;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class TestCompactionUtils {
|
||||
|
||||
private static String TEST_WRITE_TOKEN = "1-0-1";
|
||||
|
||||
private static final Map<String, Double> metrics =
|
||||
new ImmutableMap.Builder<String, Double>()
|
||||
.put("key1", 1.0)
|
||||
.put("key2", 3.0).build();
|
||||
@Rule
|
||||
public TemporaryFolder tmpFolder = new TemporaryFolder();
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private String basePath;
|
||||
private Function<Pair<String, FileSlice>, Map<String, Double>> metricsCaptureFn = (partitionFileSlice) -> metrics;
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
metaClient = HoodieTestUtils.initTableType(getDefaultHadoopConf(),
|
||||
tmpFolder.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
|
||||
basePath = metaClient.getBasePath();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBuildFromFileSlice() {
|
||||
// Empty File-Slice with no data and log files
|
||||
FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "empty1");
|
||||
HoodieCompactionOperation op = CompactionUtils.buildFromFileSlice(
|
||||
DEFAULT_PARTITION_PATHS[0], emptyFileSlice, Option.of(metricsCaptureFn));
|
||||
testFileSliceCompactionOpEquality(emptyFileSlice, op, DEFAULT_PARTITION_PATHS[0]);
|
||||
|
||||
// File Slice with data-file but no log files
|
||||
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noLog1");
|
||||
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
op = CompactionUtils.buildFromFileSlice(
|
||||
DEFAULT_PARTITION_PATHS[0], noLogFileSlice, Option.of(metricsCaptureFn));
|
||||
testFileSliceCompactionOpEquality(noLogFileSlice, op, DEFAULT_PARTITION_PATHS[0]);
|
||||
|
||||
//File Slice with no data-file but log files present
|
||||
FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN))));
|
||||
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN))));
|
||||
op = CompactionUtils.buildFromFileSlice(
|
||||
DEFAULT_PARTITION_PATHS[0], noDataFileSlice, Option.of(metricsCaptureFn));
|
||||
testFileSliceCompactionOpEquality(noDataFileSlice, op, DEFAULT_PARTITION_PATHS[0]);
|
||||
|
||||
//File Slice with data-file and log files present
|
||||
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN))));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN))));
|
||||
op = CompactionUtils.buildFromFileSlice(
|
||||
DEFAULT_PARTITION_PATHS[0], fileSlice, Option.of(metricsCaptureFn));
|
||||
testFileSliceCompactionOpEquality(fileSlice, op, DEFAULT_PARTITION_PATHS[0]);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate input for compaction plan tests
|
||||
*/
|
||||
private Pair<List<Pair<String, FileSlice>>, HoodieCompactionPlan> buildCompactionPlan() {
|
||||
FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "empty1");
|
||||
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN))));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN))));
|
||||
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noLog1");
|
||||
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN))));
|
||||
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN))));
|
||||
List<FileSlice> fileSliceList = Arrays.asList(emptyFileSlice, noDataFileSlice, fileSlice, noLogFileSlice);
|
||||
List<Pair<String, FileSlice>> input = fileSliceList.stream().map(f -> Pair.of(DEFAULT_PARTITION_PATHS[0], f))
|
||||
.collect(Collectors.toList());
|
||||
return Pair.of(input, CompactionUtils.buildFromFileSlices(input, Option.empty(), Option.of(metricsCaptureFn)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBuildFromFileSlices() {
|
||||
Pair<List<Pair<String, FileSlice>>, HoodieCompactionPlan> inputAndPlan = buildCompactionPlan();
|
||||
testFileSlicesCompactionPlanEquality(inputAndPlan.getKey(), inputAndPlan.getValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompactionTransformation() {
|
||||
// check HoodieCompactionOperation <=> CompactionOperation transformation function
|
||||
Pair<List<Pair<String, FileSlice>>, HoodieCompactionPlan> inputAndPlan = buildCompactionPlan();
|
||||
HoodieCompactionPlan plan = inputAndPlan.getRight();
|
||||
List<HoodieCompactionOperation> originalOps = plan.getOperations();
|
||||
List<HoodieCompactionOperation> regeneratedOps =
|
||||
originalOps.stream().map(op -> {
|
||||
// Convert to CompactionOperation
|
||||
return CompactionUtils.buildCompactionOperation(op);
|
||||
}).map(op2 -> {
|
||||
// Convert back to HoodieCompactionOperation and check for equality
|
||||
return CompactionUtils.buildHoodieCompactionOperation(op2);
|
||||
}).collect(Collectors.toList());
|
||||
Assert.assertTrue("Transformation did get tested", originalOps.size() > 0);
|
||||
Assert.assertEquals("All fields set correctly in transformations", originalOps, regeneratedOps);
|
||||
}
|
||||
|
||||
@Test(expected = IllegalStateException.class)
|
||||
public void testGetAllPendingCompactionOperationsWithDupFileId() throws IOException {
|
||||
// Case where there is duplicate fileIds in compaction requests
|
||||
HoodieCompactionPlan plan1 = createCompactionPlan(metaClient, "000", "001", 10, true, true);
|
||||
HoodieCompactionPlan plan2 = createCompactionPlan(metaClient, "002", "003", 0, false, false);
|
||||
scheduleCompaction(metaClient, "001", plan1);
|
||||
scheduleCompaction(metaClient, "003", plan2);
|
||||
// schedule same plan again so that there will be duplicates
|
||||
scheduleCompaction(metaClient, "005", plan1);
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> res =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetAllPendingCompactionOperations() throws IOException {
|
||||
// Case where there are 4 compaction requests where 1 is empty.
|
||||
setupAndValidateCompactionOperations(metaClient, false, 10, 10, 10, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetAllPendingInflightCompactionOperations() throws IOException {
|
||||
// Case where there are 4 compaction requests where 1 is empty. All of them are marked inflight
|
||||
setupAndValidateCompactionOperations(metaClient, true, 10, 10, 10, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetAllPendingCompactionOperationsForEmptyCompactions() throws IOException {
|
||||
// Case where there are 4 compaction requests and all are empty.
|
||||
setupAndValidateCompactionOperations(metaClient, false, 0, 0, 0, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates if generated compaction plan matches with input file-slices
|
||||
*
|
||||
* @param input File Slices with partition-path
|
||||
* @param plan Compaction Plan
|
||||
*/
|
||||
private void testFileSlicesCompactionPlanEquality(List<Pair<String, FileSlice>> input,
|
||||
HoodieCompactionPlan plan) {
|
||||
Assert.assertEquals("All file-slices present", input.size(), plan.getOperations().size());
|
||||
IntStream.range(0, input.size()).boxed().forEach(idx ->
|
||||
testFileSliceCompactionOpEquality(input.get(idx).getValue(), plan.getOperations().get(idx),
|
||||
input.get(idx).getKey()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates if generated compaction operation matches with input file slice and partition path
|
||||
*
|
||||
* @param slice File Slice
|
||||
* @param op HoodieCompactionOperation
|
||||
* @param expPartitionPath Partition path
|
||||
*/
|
||||
private void testFileSliceCompactionOpEquality(FileSlice slice, HoodieCompactionOperation op,
|
||||
String expPartitionPath) {
|
||||
Assert.assertEquals("Partition path is correct", expPartitionPath, op.getPartitionPath());
|
||||
Assert.assertEquals("Same base-instant", slice.getBaseInstantTime(), op.getBaseInstantTime());
|
||||
Assert.assertEquals("Same file-id", slice.getFileId(), op.getFileId());
|
||||
if (slice.getDataFile().isPresent()) {
|
||||
Assert.assertEquals("Same data-file", slice.getDataFile().get().getPath(), op.getDataFilePath());
|
||||
}
|
||||
List<String> paths = slice.getLogFiles().map(l -> l.getPath().toString()).collect(Collectors.toList());
|
||||
IntStream.range(0, paths.size()).boxed().forEach(idx -> {
|
||||
Assert.assertEquals("Log File Index " + idx, paths.get(idx), op.getDeltaFilePaths().get(idx));
|
||||
});
|
||||
Assert.assertEquals("Metrics set", metrics, op.getMetrics());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,138 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hudi.common.minicluster.HdfsTestService;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Tests basic functionality of {@link DFSPropertiesConfiguration}
|
||||
*/
|
||||
public class TestDFSPropertiesConfiguration {
|
||||
|
||||
private static String dfsBasePath;
|
||||
private static HdfsTestService hdfsTestService;
|
||||
private static MiniDFSCluster dfsCluster;
|
||||
private static DistributedFileSystem dfs;
|
||||
|
||||
|
||||
@BeforeClass
|
||||
public static void initClass() throws Exception {
|
||||
hdfsTestService = new HdfsTestService();
|
||||
dfsCluster = hdfsTestService.start(true);
|
||||
// Create a temp folder as the base path
|
||||
dfs = dfsCluster.getFileSystem();
|
||||
dfsBasePath = dfs.getWorkingDirectory().toString();
|
||||
dfs.mkdirs(new Path(dfsBasePath));
|
||||
|
||||
// create some files.
|
||||
Path filePath = new Path(dfsBasePath + "/t1.props");
|
||||
writePropertiesFile(filePath, new String[]{
|
||||
"", "#comment", "abc",// to be ignored
|
||||
"int.prop=123", "double.prop=113.4", "string.prop=str", "boolean.prop=true", "long.prop=1354354354"
|
||||
});
|
||||
|
||||
filePath = new Path(dfsBasePath + "/t2.props");
|
||||
writePropertiesFile(filePath, new String[]{
|
||||
"string.prop=ignored", "include=t1.props"
|
||||
});
|
||||
|
||||
filePath = new Path(dfsBasePath + "/t3.props");
|
||||
writePropertiesFile(filePath, new String[]{
|
||||
"double.prop=838.3", "include = t2.props", "double.prop=243.4", "string.prop=t3.value"
|
||||
});
|
||||
|
||||
filePath = new Path(dfsBasePath + "/t4.props");
|
||||
writePropertiesFile(filePath, new String[]{
|
||||
"double.prop=838.3", "include = t4.props"
|
||||
});
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void cleanupClass() throws Exception {
|
||||
if (hdfsTestService != null) {
|
||||
hdfsTestService.stop();
|
||||
}
|
||||
}
|
||||
|
||||
private static void writePropertiesFile(Path path, String[] lines) throws IOException {
|
||||
PrintStream out = new PrintStream(dfs.create(path, true));
|
||||
for (String line : lines) {
|
||||
out.println(line);
|
||||
}
|
||||
out.flush();
|
||||
out.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsing() throws IOException {
|
||||
DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t1.props"));
|
||||
TypedProperties props = cfg.getConfig();
|
||||
assertEquals(5, props.size());
|
||||
try {
|
||||
props.getString("invalid.key");
|
||||
fail("Should error out here.");
|
||||
} catch (IllegalArgumentException iae) { /* ignore */ }
|
||||
|
||||
assertEquals(123, props.getInteger("int.prop"));
|
||||
assertEquals(113.4, props.getDouble("double.prop"), 0.001);
|
||||
assertEquals(true, props.getBoolean("boolean.prop"));
|
||||
assertEquals("str", props.getString("string.prop"));
|
||||
assertEquals(1354354354, props.getLong("long.prop"));
|
||||
|
||||
assertEquals(123, props.getInteger("int.prop", 456));
|
||||
assertEquals(113.4, props.getDouble("double.prop", 223.4), 0.001);
|
||||
assertEquals(true, props.getBoolean("boolean.prop", false));
|
||||
assertEquals("str", props.getString("string.prop", "default"));
|
||||
assertEquals(1354354354, props.getLong("long.prop", 8578494434L));
|
||||
|
||||
assertEquals(456, props.getInteger("bad.int.prop", 456));
|
||||
assertEquals(223.4, props.getDouble("bad.double.prop", 223.4), 0.001);
|
||||
assertEquals(false, props.getBoolean("bad.boolean.prop", false));
|
||||
assertEquals("default", props.getString("bad.string.prop", "default"));
|
||||
assertEquals(8578494434L, props.getLong("bad.long.prop", 8578494434L));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIncludes() {
|
||||
DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t3.props"));
|
||||
TypedProperties props = cfg.getConfig();
|
||||
|
||||
assertEquals(123, props.getInteger("int.prop"));
|
||||
assertEquals(243.4, props.getDouble("double.prop"), 0.001);
|
||||
assertEquals(true, props.getBoolean("boolean.prop"));
|
||||
assertEquals("t3.value", props.getString("string.prop"));
|
||||
assertEquals(1354354354, props.getLong("long.prop"));
|
||||
|
||||
try {
|
||||
new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t4.props"));
|
||||
fail("Should error out on a self-included file.");
|
||||
} catch (IllegalStateException ise) { /* ignore */ }
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,273 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.contrib.java.lang.system.EnvironmentVariables;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class TestFSUtils {
|
||||
|
||||
private static String TEST_WRITE_TOKEN = "1-0-1";
|
||||
|
||||
@Rule
|
||||
public final EnvironmentVariables environmentVariables = new EnvironmentVariables();
|
||||
|
||||
@Test
|
||||
public void testMakeDataFileName() {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
int taskPartitionId = 2;
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
assertTrue(FSUtils.makeDataFileName(commitTime, TEST_WRITE_TOKEN, fileName)
|
||||
.equals(fileName + "_" + TEST_WRITE_TOKEN + "_" + commitTime + ".parquet"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMaskFileName() {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
int taskPartitionId = 2;
|
||||
assertTrue(FSUtils.maskWithoutFileId(commitTime, taskPartitionId)
|
||||
.equals("*_" + taskPartitionId + "_" + commitTime + ".parquet"));
|
||||
}
|
||||
|
||||
@Test
|
||||
/**
|
||||
* Tests if process Files return only paths excluding marker directories
|
||||
* Cleaner, Rollback and compaction-scheduling logic was recursively processing all subfolders including that
|
||||
* of ".hoodie" when looking for partition-paths. This causes a race when they try to list all folders (recursively)
|
||||
* but the marker directory (that of compaction inside of ".hoodie" folder) is deleted underneath by compactor.
|
||||
* This code tests the fix by ensuring ".hoodie" and their subfolders are never processed.
|
||||
*/
|
||||
public void testProcessFiles() throws Exception {
|
||||
TemporaryFolder tmpFolder = new TemporaryFolder();
|
||||
tmpFolder.create();
|
||||
// All directories including marker dirs.
|
||||
List<String> folders = Arrays.asList("2016/04/15", "2016/05/16", ".hoodie/.temp/2/2016/04/15",
|
||||
".hoodie/.temp/2/2016/05/16");
|
||||
HoodieTableMetaClient metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath());
|
||||
String basePath = metaClient.getBasePath();
|
||||
folders.stream().forEach(f -> {
|
||||
try {
|
||||
metaClient.getFs().mkdirs(new Path(new Path(basePath), f));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
|
||||
// Files inside partitions and marker directories
|
||||
List<String> files = Arrays.asList(
|
||||
"2016/04/15/1_1-0-1_20190528120000.parquet",
|
||||
"2016/05/16/2_1-0-1_20190528120000.parquet",
|
||||
".hoodie/.temp/2/2016/05/16/2_1-0-1_20190528120000.parquet",
|
||||
".hoodie/.temp/2/2016/04/15/1_1-0-1_20190528120000.parquet"
|
||||
);
|
||||
|
||||
files.stream().forEach(f -> {
|
||||
try {
|
||||
metaClient.getFs().create(new Path(new Path(basePath), f));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
|
||||
// Test excluding meta-folder
|
||||
final List<String> collected = new ArrayList<>();
|
||||
FSUtils.processFiles(metaClient.getFs(), basePath, (status) -> {
|
||||
collected.add(status.getPath().toString());
|
||||
return true;
|
||||
}, true);
|
||||
|
||||
Assert.assertTrue("Hoodie MetaFolder MUST be skipped but got :" + collected, collected.stream()
|
||||
.noneMatch(s -> s.contains(HoodieTableMetaClient.METAFOLDER_NAME)));
|
||||
// Check if only files are listed
|
||||
Assert.assertEquals(2, collected.size());
|
||||
|
||||
// Test including meta-folder
|
||||
final List<String> collected2 = new ArrayList<>();
|
||||
FSUtils.processFiles(metaClient.getFs(), basePath, (status) -> {
|
||||
collected2.add(status.getPath().toString());
|
||||
return true;
|
||||
}, false);
|
||||
|
||||
Assert.assertFalse("Hoodie MetaFolder will be present :" + collected2, collected2.stream()
|
||||
.noneMatch(s -> s.contains(HoodieTableMetaClient.METAFOLDER_NAME)));
|
||||
// Check if only files are listed including hoodie.properties
|
||||
Assert.assertEquals("Collected=" + collected2, 5, collected2.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetCommitTime() {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
int taskPartitionId = 2;
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String fullFileName = FSUtils.makeDataFileName(commitTime, TEST_WRITE_TOKEN, fileName);
|
||||
assertTrue(FSUtils.getCommitTime(fullFileName).equals(commitTime));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetFileNameWithoutMeta() {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
int taskPartitionId = 2;
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String fullFileName = FSUtils.makeDataFileName(commitTime, TEST_WRITE_TOKEN, fileName);
|
||||
assertTrue(FSUtils.getFileId(fullFileName).equals(fileName));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEnvVarVariablesPickedup() {
|
||||
environmentVariables.set("HOODIE_ENV_fs_DOT_key1", "value1");
|
||||
Configuration conf = FSUtils.prepareHadoopConf(HoodieTestUtils.getDefaultHadoopConf());
|
||||
assertEquals("value1", conf.get("fs.key1"));
|
||||
conf.set("fs.key1", "value11");
|
||||
conf.set("fs.key2", "value2");
|
||||
assertEquals("value11", conf.get("fs.key1"));
|
||||
assertEquals("value2", conf.get("fs.key2"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetRelativePartitionPath() {
|
||||
Path basePath = new Path("/test/apache");
|
||||
Path partitionPath = new Path("/test/apache/hudi/sub");
|
||||
assertEquals("hudi/sub",FSUtils.getRelativePartitionPath(basePath, partitionPath));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetRelativePartitionPathSameFolder() {
|
||||
Path basePath = new Path("/test");
|
||||
Path partitionPath = new Path("/test");
|
||||
assertEquals("", FSUtils.getRelativePartitionPath(basePath, partitionPath));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetRelativePartitionPathRepeatedFolderNameBasePath() {
|
||||
Path basePath = new Path("/test/apache/apache");
|
||||
Path partitionPath = new Path("/test/apache/apache/hudi");
|
||||
assertEquals("hudi", FSUtils.getRelativePartitionPath(basePath, partitionPath));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetRelativePartitionPathRepeatedFolderNamePartitionPath() {
|
||||
Path basePath = new Path("/test/apache");
|
||||
Path partitionPath = new Path("/test/apache/apache/hudi");
|
||||
assertEquals("apache/hudi", FSUtils.getRelativePartitionPath(basePath, partitionPath));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOldLogFileName() {
|
||||
// Check if old log file names are still parseable by FSUtils method
|
||||
String partitionPath = "2019/01/01/";
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String oldLogFile = makeOldLogFileName(fileName, ".log", "100", 1);
|
||||
Path rlPath = new Path(new Path(partitionPath), oldLogFile);
|
||||
Assert.assertTrue(FSUtils.isLogFile(rlPath));
|
||||
assertEquals(fileName, FSUtils.getFileIdFromLogPath(rlPath));
|
||||
assertEquals("100", FSUtils.getBaseCommitTimeFromLogPath(rlPath));
|
||||
assertEquals(1, FSUtils.getFileVersionFromLog(rlPath));
|
||||
Assert.assertNull(FSUtils.getTaskPartitionIdFromLogPath(rlPath));
|
||||
Assert.assertNull(FSUtils.getStageIdFromLogPath(rlPath));
|
||||
Assert.assertNull(FSUtils.getTaskAttemptIdFromLogPath(rlPath));
|
||||
Assert.assertNull(FSUtils.getWriteTokenFromLogPath(rlPath));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void tesLogFileName() {
|
||||
// Check if log file names are parseable by FSUtils method
|
||||
String partitionPath = "2019/01/01/";
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String logFile = FSUtils.makeLogFileName(fileName, ".log", "100", 2, "1-0-1");
|
||||
System.out.println("Log File =" + logFile);
|
||||
Path rlPath = new Path(new Path(partitionPath), logFile);
|
||||
Assert.assertTrue(FSUtils.isLogFile(rlPath));
|
||||
assertEquals(fileName, FSUtils.getFileIdFromLogPath(rlPath));
|
||||
assertEquals("100", FSUtils.getBaseCommitTimeFromLogPath(rlPath));
|
||||
assertEquals(2, FSUtils.getFileVersionFromLog(rlPath));
|
||||
assertEquals(new Integer(1), FSUtils.getTaskPartitionIdFromLogPath(rlPath));
|
||||
assertEquals(new Integer(0), FSUtils.getStageIdFromLogPath(rlPath));
|
||||
assertEquals(new Integer(1), FSUtils.getTaskAttemptIdFromLogPath(rlPath));
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Log File Comparisons when log files do not have write tokens.
|
||||
*/
|
||||
@Test
|
||||
public void testOldLogFilesComparison() {
|
||||
String log1Ver0 = makeOldLogFileName("file1", ".log", "1", 0);
|
||||
String log1Ver1 = makeOldLogFileName("file1", ".log", "1", 1);
|
||||
String log1base2 = makeOldLogFileName("file1", ".log", "2", 0);
|
||||
List<HoodieLogFile> logFiles =
|
||||
Arrays.asList(log1base2, log1Ver1, log1Ver0).stream()
|
||||
.map(f -> new HoodieLogFile(f)).collect(Collectors.toList());
|
||||
logFiles.sort(HoodieLogFile.getLogFileComparator());
|
||||
assertEquals(log1Ver0, logFiles.get(0).getFileName());
|
||||
assertEquals(log1Ver1, logFiles.get(1).getFileName());
|
||||
assertEquals(log1base2, logFiles.get(2).getFileName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Log File Comparisons when log files do not have write tokens.
|
||||
*/
|
||||
@Test
|
||||
public void testLogFilesComparison() {
|
||||
String log1Ver0W0 = FSUtils.makeLogFileName("file1", ".log", "1", 0, "0-0-1");
|
||||
String log1Ver0W1 = FSUtils.makeLogFileName("file1", ".log", "1", 0, "1-1-1");
|
||||
String log1Ver1W0 = FSUtils.makeLogFileName("file1", ".log", "1", 1, "0-0-1");
|
||||
String log1Ver1W1 = FSUtils.makeLogFileName("file1", ".log", "1", 1, "1-1-1");
|
||||
String log1base2W0 = FSUtils.makeLogFileName("file1", ".log", "2", 0, "0-0-1");
|
||||
String log1base2W1 = FSUtils.makeLogFileName("file1", ".log", "2", 0, "1-1-1");
|
||||
|
||||
List<HoodieLogFile> logFiles =
|
||||
Arrays.asList(log1Ver1W1, log1base2W0, log1base2W1, log1Ver1W0, log1Ver0W1, log1Ver0W0).stream()
|
||||
.map(f -> new HoodieLogFile(f)).collect(Collectors.toList());
|
||||
logFiles.sort(HoodieLogFile.getLogFileComparator());
|
||||
assertEquals(log1Ver0W0, logFiles.get(0).getFileName());
|
||||
assertEquals(log1Ver0W1, logFiles.get(1).getFileName());
|
||||
assertEquals(log1Ver1W0, logFiles.get(2).getFileName());
|
||||
assertEquals(log1Ver1W1, logFiles.get(3).getFileName());
|
||||
assertEquals(log1base2W0, logFiles.get(4).getFileName());
|
||||
assertEquals(log1base2W1, logFiles.get(5).getFileName());
|
||||
}
|
||||
|
||||
public static String makeOldLogFileName(String fileId, String logFileExtension,
|
||||
String baseCommitTime, int version) {
|
||||
Pattern oldLogFilePattern =
|
||||
Pattern.compile("\\.(.*)_(.*)\\.(.*)\\.([0-9]*)(\\.([0-9]*))");
|
||||
return "." + String
|
||||
.format("%s_%s%s.%d", fileId, baseCommitTime, logFileExtension, version);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,58 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import java.util.Map;
|
||||
import org.apache.avro.Schema;
|
||||
import org.codehaus.jackson.JsonNode;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
|
||||
public class TestHoodieAvroUtils {
|
||||
|
||||
private static String EXAMPLE_SCHEMA = "{\"type\": \"record\"," + "\"name\": \"testrec\"," + "\"fields\": [ "
|
||||
+ "{\"name\": \"timestamp\",\"type\": \"double\"},"
|
||||
+ "{\"name\": \"_row_key\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"non_pii_col\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"pii_col\", \"type\": \"string\", \"column_category\": \"user_profile\"}]}";
|
||||
|
||||
@Test
|
||||
public void testPropsPresent() {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(EXAMPLE_SCHEMA));
|
||||
boolean piiPresent = false;
|
||||
for (Schema.Field field : schema.getFields()) {
|
||||
if (HoodieAvroUtils.isMetadataField(field.name())) {
|
||||
continue;
|
||||
}
|
||||
|
||||
Assert.assertTrue("field name is null", field.name() != null);
|
||||
Map<String, JsonNode> props = field.getJsonProps();
|
||||
Assert.assertTrue("The property is null", props != null);
|
||||
|
||||
if (field.name().equals("pii_col")) {
|
||||
piiPresent = true;
|
||||
Assert.assertTrue("sensitivity_level is removed in field 'pii_col'", props.containsKey("column_category"));
|
||||
} else {
|
||||
Assert.assertTrue("The property shows up but not set", props.size() == 0);
|
||||
}
|
||||
}
|
||||
Assert.assertTrue("column pii_col doesn't show up", piiPresent);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,39 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestNumericUtils {
|
||||
|
||||
@Test
|
||||
public void testHumanReadableByteCount() {
|
||||
assertTrue(NumericUtils.humanReadableByteCount(0).equals("0.0 B"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(27).equals("27.0 B"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(1023).equals("1023.0 B"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(1024).equals("1.0 KB"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(110592).equals("108.0 KB"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(28991029248L).equals("27.0 GB"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(1855425871872L).equals("1.7 TB"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(9223372036854775807L).equals("8.0 EB"));
|
||||
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,127 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.common.BloomFilter;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class TestParquetUtils {
|
||||
|
||||
|
||||
private String basePath;
|
||||
|
||||
@Before
|
||||
public void setup() throws IOException {
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHoodieWriteSupport() throws Exception {
|
||||
List<String> rowKeys = new ArrayList<>();
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
rowKeys.add(UUID.randomUUID().toString());
|
||||
}
|
||||
|
||||
String filePath = basePath + "/test.parquet";
|
||||
writeParquetFile(filePath, rowKeys);
|
||||
|
||||
// Read and verify
|
||||
List<String> rowKeysInFile = new ArrayList<>(
|
||||
ParquetUtils.readRowKeysFromParquet(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath)));
|
||||
Collections.sort(rowKeysInFile);
|
||||
Collections.sort(rowKeys);
|
||||
|
||||
assertEquals("Did not read back the expected list of keys", rowKeys, rowKeysInFile);
|
||||
BloomFilter filterInFile = ParquetUtils.readBloomFilterFromParquetMetadata(HoodieTestUtils.getDefaultHadoopConf(),
|
||||
new Path(filePath));
|
||||
for (String rowKey : rowKeys) {
|
||||
assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterParquetRowKeys() throws Exception {
|
||||
List<String> rowKeys = new ArrayList<>();
|
||||
Set<String> filter = new HashSet<>();
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
String rowKey = UUID.randomUUID().toString();
|
||||
rowKeys.add(rowKey);
|
||||
if (i % 100 == 0) {
|
||||
filter.add(rowKey);
|
||||
}
|
||||
}
|
||||
|
||||
String filePath = basePath + "/test.parquet";
|
||||
writeParquetFile(filePath, rowKeys);
|
||||
|
||||
// Read and verify
|
||||
Set<String> filtered = ParquetUtils.filterParquetRowKeys(HoodieTestUtils.getDefaultHadoopConf(),
|
||||
new Path(filePath),
|
||||
filter);
|
||||
|
||||
assertEquals("Filtered count does not match", filter.size(), filtered.size());
|
||||
|
||||
for (String rowKey : filtered) {
|
||||
assertTrue("filtered key must be in the given filter", filter.contains(rowKey));
|
||||
}
|
||||
}
|
||||
|
||||
private void writeParquetFile(String filePath,
|
||||
List<String> rowKeys) throws Exception {
|
||||
// Write out a parquet file
|
||||
Schema schema = HoodieAvroUtils.getRecordKeySchema();
|
||||
BloomFilter filter = new BloomFilter(1000, 0.0001);
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema,
|
||||
filter);
|
||||
ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP,
|
||||
120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE);
|
||||
for (String rowKey : rowKeys) {
|
||||
GenericRecord rec = new GenericData.Record(schema);
|
||||
rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey);
|
||||
writer.write(rec);
|
||||
filter.add(rowKey);
|
||||
}
|
||||
writer.close();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,105 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonAutoDetect;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.PropertyAccessor;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
@SuppressWarnings({"unused", "FieldCanBeLocal", "MismatchedQueryAndUpdateOfCollection"})
|
||||
public class TestRecord implements Serializable {
|
||||
|
||||
class TestMapItemRecord implements Serializable {
|
||||
|
||||
private String item1;
|
||||
private String item2;
|
||||
|
||||
TestMapItemRecord(String item1, String item2) {
|
||||
this.item1 = item1;
|
||||
this.item2 = item2;
|
||||
}
|
||||
}
|
||||
|
||||
class TestNestedRecord implements Serializable {
|
||||
|
||||
private boolean isAdmin;
|
||||
private String userId;
|
||||
|
||||
TestNestedRecord(boolean isAdmin, String userId) {
|
||||
this.isAdmin = isAdmin;
|
||||
this.userId = userId;
|
||||
}
|
||||
}
|
||||
|
||||
private String _hoodie_commit_time;
|
||||
private String _hoodie_record_key;
|
||||
private String _hoodie_partition_path;
|
||||
private String _hoodie_file_name;
|
||||
private String _hoodie_commit_seqno;
|
||||
|
||||
private String field1;
|
||||
private String field2;
|
||||
private String name;
|
||||
private Integer favoriteIntNumber;
|
||||
private Long favoriteNumber;
|
||||
private Float favoriteFloatNumber;
|
||||
private Double favoriteDoubleNumber;
|
||||
private Map<String, TestMapItemRecord> tags;
|
||||
private TestNestedRecord testNestedRecord;
|
||||
private String[] stringArray;
|
||||
|
||||
public TestRecord(String commitTime, int recordNumber, String fileId) {
|
||||
this._hoodie_commit_time = commitTime;
|
||||
this._hoodie_record_key = "key" + recordNumber;
|
||||
this._hoodie_partition_path = commitTime;
|
||||
this._hoodie_file_name = fileId;
|
||||
this._hoodie_commit_seqno = commitTime + recordNumber;
|
||||
|
||||
String commitTimeSuffix = "@" + commitTime;
|
||||
int commitHashCode = commitTime.hashCode();
|
||||
|
||||
this.field1 = "field" + recordNumber;
|
||||
this.field2 = "field" + recordNumber + commitTimeSuffix;
|
||||
this.name = "name" + recordNumber;
|
||||
this.favoriteIntNumber = recordNumber + commitHashCode;
|
||||
this.favoriteNumber = (long) (recordNumber + commitHashCode);
|
||||
this.favoriteFloatNumber = (float) ((recordNumber + commitHashCode) / 1024.0);
|
||||
this.favoriteDoubleNumber = (recordNumber + commitHashCode) / 1024.0;
|
||||
this.tags = new HashMap<>();
|
||||
this.tags.put("mapItem1",
|
||||
new TestMapItemRecord("item" + recordNumber, "item" + recordNumber + commitTimeSuffix));
|
||||
this.tags.put("mapItem2",
|
||||
new TestMapItemRecord("item2" + recordNumber, "item2" + recordNumber + commitTimeSuffix));
|
||||
this.testNestedRecord = new TestNestedRecord(false, "UserId" + recordNumber + commitTimeSuffix);
|
||||
this.stringArray = new String[]{"stringArray0" + commitTimeSuffix,
|
||||
"stringArray1" + commitTimeSuffix};
|
||||
}
|
||||
|
||||
public String toJsonString() throws IOException {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY);
|
||||
return mapper.writerWithDefaultPrettyPrinter().writeValueAsString(this);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,180 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestRocksDBManager {
|
||||
|
||||
private static RocksDBDAO dbManager;
|
||||
|
||||
@AfterClass
|
||||
public static void drop() throws IOException {
|
||||
if (dbManager != null) {
|
||||
dbManager.close();
|
||||
dbManager = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRocksDBManager() throws Exception {
|
||||
String prefix1 = "prefix1_";
|
||||
String prefix2 = "prefix2_";
|
||||
String prefix3 = "prefix3_";
|
||||
String prefix4 = "prefix4_";
|
||||
List<String> prefixes = Arrays.asList(prefix1, prefix2, prefix3, prefix4);
|
||||
String family1 = "family1";
|
||||
String family2 = "family2";
|
||||
List<String> colFamilies = Arrays.asList(family1, family2);
|
||||
|
||||
List<Payload> payloads = IntStream.range(0, 100).mapToObj(index -> {
|
||||
String prefix = prefixes.get(index % 4);
|
||||
String key = prefix + UUID.randomUUID().toString();
|
||||
String family = colFamilies.get(index % 2);
|
||||
String val = "VALUE_" + UUID.randomUUID().toString();
|
||||
return new Payload(prefix, key, val, family);
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
dbManager = new RocksDBDAO("/dummy/path",
|
||||
FileSystemViewStorageConfig.newBuilder().build().newBuilder().build().getRocksdbBasePath());
|
||||
colFamilies.stream().forEach(family -> dbManager.dropColumnFamily(family));
|
||||
colFamilies.stream().forEach(family -> dbManager.addColumnFamily(family));
|
||||
|
||||
Map<String, Map<String, Integer>> countsMap = new HashMap<>();
|
||||
payloads.stream().forEach(payload -> {
|
||||
dbManager.put(payload.getFamily(), payload.getKey(), payload);
|
||||
|
||||
if (!countsMap.containsKey(payload.family)) {
|
||||
countsMap.put(payload.family, new HashMap<>());
|
||||
}
|
||||
Map<String, Integer> c = countsMap.get(payload.family);
|
||||
if (!c.containsKey(payload.prefix)) {
|
||||
c.put(payload.prefix, 0);
|
||||
}
|
||||
int currCount = c.get(payload.prefix);
|
||||
c.put(payload.prefix, currCount + 1);
|
||||
});
|
||||
|
||||
colFamilies.stream().forEach(family -> {
|
||||
prefixes.stream().forEach(prefix -> {
|
||||
List<Pair<String, Payload>> gotPayloads =
|
||||
dbManager.<Payload>prefixSearch(family, prefix).collect(Collectors.toList());
|
||||
Integer expCount = countsMap.get(family).get(prefix);
|
||||
Assert.assertEquals("Size check for prefix (" + prefix + ") and family (" + family + ")",
|
||||
expCount == null ? 0L : expCount.longValue(), gotPayloads.size());
|
||||
gotPayloads.stream().forEach(p -> {
|
||||
Assert.assertEquals(p.getRight().getFamily(), family);
|
||||
Assert.assertTrue(p.getRight().getKey().startsWith(prefix));
|
||||
});
|
||||
});
|
||||
});
|
||||
|
||||
payloads.stream().forEach(payload -> {
|
||||
Payload p = dbManager.get(payload.getFamily(), payload.getKey());
|
||||
Assert.assertEquals("Retrieved correct payload for key :" + payload.getKey(), payload, p);
|
||||
|
||||
// Now, delete the key
|
||||
dbManager.delete(payload.getFamily(), payload.getKey());
|
||||
|
||||
// Now retrieve
|
||||
Payload p2 = dbManager.get(payload.getFamily(), payload.getKey());
|
||||
Assert.assertNull("Retrieved correct payload for key :" + p.getKey(), p2);
|
||||
});
|
||||
|
||||
// Now do a prefix search
|
||||
colFamilies.stream().forEach(family -> {
|
||||
prefixes.stream().forEach(prefix -> {
|
||||
List<Pair<String, Payload>> gotPayloads =
|
||||
dbManager.<Payload>prefixSearch(family, prefix).collect(Collectors.toList());
|
||||
Assert.assertEquals("Size check for prefix (" + prefix + ") and family (" + family + ")",
|
||||
0, gotPayloads.size());
|
||||
});
|
||||
});
|
||||
|
||||
String rocksDBBasePath = dbManager.getRocksDBBasePath();
|
||||
dbManager.close();
|
||||
Assert.assertFalse(new File(rocksDBBasePath).exists());
|
||||
}
|
||||
|
||||
public static class Payload implements Serializable {
|
||||
|
||||
private final String prefix;
|
||||
private final String key;
|
||||
private final String val;
|
||||
private final String family;
|
||||
|
||||
public Payload(String prefix, String key, String val, String family) {
|
||||
this.prefix = prefix;
|
||||
this.key = key;
|
||||
this.val = val;
|
||||
this.family = family;
|
||||
}
|
||||
|
||||
public String getPrefix() {
|
||||
return prefix;
|
||||
}
|
||||
|
||||
public String getKey() {
|
||||
return key;
|
||||
}
|
||||
|
||||
public String getVal() {
|
||||
return val;
|
||||
}
|
||||
|
||||
public String getFamily() {
|
||||
return family;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
Payload payload = (Payload) o;
|
||||
return Objects.equals(prefix, payload.prefix)
|
||||
&& Objects.equals(key, payload.key)
|
||||
&& Objects.equals(val, payload.val)
|
||||
&& Objects.equals(family, payload.family);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(prefix, key, val, family);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,81 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.LinkedList;
|
||||
import java.util.Objects;
|
||||
import org.apache.avro.util.Utf8;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestSerializationUtils {
|
||||
|
||||
@Test
|
||||
public void testSerDeser() throws IOException {
|
||||
// It should handle null object references.
|
||||
verifyObject(null);
|
||||
// Object with nulls.
|
||||
verifyObject(new NonSerializableClass(null));
|
||||
// Object with valid values & no default constructor.
|
||||
verifyObject(new NonSerializableClass("testValue"));
|
||||
// Object with multiple constructor
|
||||
verifyObject(new NonSerializableClass("testValue1", "testValue2"));
|
||||
// Object which is of non-serializable class.
|
||||
verifyObject(new Utf8("test-key"));
|
||||
// Verify serialization of list.
|
||||
verifyObject(new LinkedList<>(Arrays.asList(2, 3, 5)));
|
||||
}
|
||||
|
||||
private <T> void verifyObject(T expectedValue) throws IOException {
|
||||
byte[] serializedObject = SerializationUtils.serialize(expectedValue);
|
||||
Assert.assertTrue(serializedObject != null && serializedObject.length > 0);
|
||||
|
||||
final T deserializedValue = SerializationUtils.<T>deserialize(serializedObject);
|
||||
if (expectedValue == null) {
|
||||
Assert.assertNull(deserializedValue);
|
||||
} else {
|
||||
Assert.assertTrue(expectedValue.equals(deserializedValue));
|
||||
}
|
||||
}
|
||||
|
||||
private static class NonSerializableClass {
|
||||
private String id;
|
||||
private String name;
|
||||
|
||||
NonSerializableClass(String id) {
|
||||
this(id, "");
|
||||
}
|
||||
|
||||
NonSerializableClass(String id, String name) {
|
||||
this.id = id;
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (!(obj instanceof NonSerializableClass)) {
|
||||
return false;
|
||||
}
|
||||
final NonSerializableClass other = (NonSerializableClass) obj;
|
||||
return Objects.equals(this.id, other.id) && Objects.equals(this.name, other.name);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,211 @@
|
||||
/*
|
||||
* 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.util.collection;
|
||||
|
||||
import static org.apache.hudi.common.util.SchemaTestUtil.getSimpleSchema;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hudi.common.model.AvroBinaryTestPayload;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.HoodieRecordSizeEstimator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.SchemaTestUtil;
|
||||
import org.apache.hudi.common.util.SpillableMapTestUtils;
|
||||
import org.apache.hudi.common.util.SpillableMapUtils;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestDiskBasedMap {
|
||||
|
||||
private static final String BASE_OUTPUT_PATH = "/tmp/";
|
||||
|
||||
@Test
|
||||
public void testSimpleInsert() throws IOException, URISyntaxException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
DiskBasedMap records = new DiskBasedMap<>(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
|
||||
assertTrue(records.sizeOfFileOnDiskInBytes() > 0);
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleInsertWithoutHoodieMetadata() throws IOException, URISyntaxException {
|
||||
Schema schema = getSimpleSchema();
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
DiskBasedMap records = new DiskBasedMap<>(BASE_OUTPUT_PATH);
|
||||
List<HoodieRecord> hoodieRecords = SchemaTestUtil
|
||||
.generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000);
|
||||
Set<String> recordKeys = new HashSet<>();
|
||||
// insert generated records into the map
|
||||
hoodieRecords.stream().forEach(r -> {
|
||||
records.put(r.getRecordKey(), r);
|
||||
recordKeys.add(r.getRecordKey());
|
||||
});
|
||||
// make sure records have spilled to disk
|
||||
assertTrue(records.sizeOfFileOnDiskInBytes() > 0);
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleUpsert() throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
DiskBasedMap records = new DiskBasedMap<>(BASE_OUTPUT_PATH);
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
|
||||
// perform some inserts
|
||||
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
|
||||
|
||||
long fileSize = records.sizeOfFileOnDiskInBytes();
|
||||
// make sure records have spilled to disk
|
||||
assertTrue(fileSize > 0);
|
||||
|
||||
// generate updates from inserts
|
||||
List<IndexedRecord> updatedRecords =
|
||||
SchemaTestUtil
|
||||
.updateHoodieTestRecords(recordKeys, SchemaTestUtil.generateHoodieTestRecords(0, 100),
|
||||
HoodieActiveTimeline.createNewCommitTime());
|
||||
String newCommitTime = ((GenericRecord) updatedRecords.get(0))
|
||||
.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
|
||||
// perform upserts
|
||||
recordKeys = SpillableMapTestUtils.upsertRecords(updatedRecords, records);
|
||||
|
||||
// upserts should be appended to the existing file, hence increasing the sizeOfFile on disk
|
||||
assertTrue(records.sizeOfFileOnDiskInBytes() > fileSize);
|
||||
|
||||
// Upserted records (on disk) should have the latest commit time
|
||||
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
|
||||
while (itr.hasNext()) {
|
||||
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
|
||||
assert recordKeys.contains(rec.getRecordKey());
|
||||
try {
|
||||
IndexedRecord indexedRecord = (IndexedRecord) rec.getData().getInsertValue(schema).get();
|
||||
String latestCommitTime = ((GenericRecord) indexedRecord)
|
||||
.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
assertEquals(latestCommitTime, newCommitTime);
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException(io);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSizeEstimator() throws IOException, URISyntaxException {
|
||||
Schema schema = SchemaTestUtil.getSimpleSchema();
|
||||
|
||||
// Test sizeEstimator without hoodie metadata fields
|
||||
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
|
||||
|
||||
long payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
|
||||
new HoodieRecordSizeEstimator(schema));
|
||||
assertTrue(payloadSize > 0);
|
||||
|
||||
// Test sizeEstimator with hoodie metadata fields
|
||||
schema = HoodieAvroUtils.addMetadataFields(schema);
|
||||
hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
|
||||
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
|
||||
new HoodieRecordSizeEstimator(schema));
|
||||
assertTrue(payloadSize > 0);
|
||||
|
||||
// Following tests payloads without an Avro Schema in the Record
|
||||
|
||||
// Test sizeEstimator without hoodie metadata fields and without schema object in the payload
|
||||
schema = SchemaTestUtil.getSimpleSchema();
|
||||
List<IndexedRecord> indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
|
||||
hoodieRecords = indexedRecords.stream()
|
||||
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
|
||||
new AvroBinaryTestPayload(Option.of((GenericRecord) r)))).collect(Collectors.toList());
|
||||
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
|
||||
new HoodieRecordSizeEstimator(schema));
|
||||
assertTrue(payloadSize > 0);
|
||||
|
||||
// Test sizeEstimator with hoodie metadata fields and without schema object in the payload
|
||||
final Schema simpleSchemaWithMetadata = HoodieAvroUtils
|
||||
.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
|
||||
hoodieRecords = indexedRecords.stream()
|
||||
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
|
||||
new AvroBinaryTestPayload(Option
|
||||
.of(HoodieAvroUtils.rewriteRecord((GenericRecord) r, simpleSchemaWithMetadata)))))
|
||||
.collect(Collectors.toList());
|
||||
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
|
||||
new HoodieRecordSizeEstimator(schema));
|
||||
assertTrue(payloadSize > 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* @na: Leaving this test here for a quick performance test
|
||||
*/
|
||||
@Ignore
|
||||
@Test
|
||||
public void testSizeEstimatorPerformance() throws IOException, URISyntaxException {
|
||||
// Test sizeEstimatorPerformance with simpleSchema
|
||||
Schema schema = SchemaTestUtil.getSimpleSchema();
|
||||
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
|
||||
HoodieRecordSizeEstimator sizeEstimator =
|
||||
new HoodieRecordSizeEstimator(schema);
|
||||
HoodieRecord record = hoodieRecords.remove(0);
|
||||
long startTime = System.currentTimeMillis();
|
||||
SpillableMapUtils.computePayloadSize(record, sizeEstimator);
|
||||
long timeTaken = System.currentTimeMillis() - startTime;
|
||||
System.out.println("Time taken :" + timeTaken);
|
||||
assertTrue(timeTaken < 100);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,320 @@
|
||||
/*
|
||||
* 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.util.collection;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.util.DefaultSizeEstimator;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.HoodieRecordSizeEstimator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.SchemaTestUtil;
|
||||
import org.apache.hudi.common.util.SpillableMapTestUtils;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.FixMethodOrder;
|
||||
import org.junit.Test;
|
||||
import org.junit.runners.MethodSorters;
|
||||
|
||||
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
|
||||
public class TestExternalSpillableMap {
|
||||
|
||||
private static final String FAILURE_OUTPUT_PATH = "/tmp/test_fail";
|
||||
private static final String BASE_OUTPUT_PATH = "/tmp/";
|
||||
|
||||
@BeforeClass
|
||||
public static void cleanUp() {
|
||||
File file = new File(BASE_OUTPUT_PATH);
|
||||
file.delete();
|
||||
file = new File(FAILURE_OUTPUT_PATH);
|
||||
file.delete();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void simpleInsertTest() throws IOException, URISyntaxException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH,
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B
|
||||
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
|
||||
assert (recordKeys.size() == 100);
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleUpsert() throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH,
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B
|
||||
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
|
||||
assert (recordKeys.size() == 100);
|
||||
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
|
||||
while (itr.hasNext()) {
|
||||
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
|
||||
assert recordKeys.contains(rec.getRecordKey());
|
||||
}
|
||||
List<IndexedRecord> updatedRecords = SchemaTestUtil.updateHoodieTestRecords(recordKeys,
|
||||
SchemaTestUtil.generateHoodieTestRecords(0, 100), HoodieActiveTimeline.createNewCommitTime());
|
||||
|
||||
// update records already inserted
|
||||
SpillableMapTestUtils.upsertRecords(updatedRecords, records);
|
||||
|
||||
// make sure we have records spilled to disk
|
||||
assertTrue(records.getDiskBasedMapNumEntries() > 0);
|
||||
|
||||
// iterate over the updated records and compare the value from Map
|
||||
updatedRecords.stream().forEach(record -> {
|
||||
HoodieRecord rec = records.get(((GenericRecord) record).get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
|
||||
try {
|
||||
assertEquals(rec.getData().getInsertValue(schema).get(), record);
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException(io);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAllMapOperations() throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH,
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B
|
||||
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
// insert a bunch of records so that values spill to disk too
|
||||
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
|
||||
IndexedRecord inMemoryRecord = iRecords.get(0);
|
||||
String ikey = ((GenericRecord) inMemoryRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String iPartitionPath = ((GenericRecord) inMemoryRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
|
||||
HoodieRecord inMemoryHoodieRecord = new HoodieRecord<>(new HoodieKey(ikey, iPartitionPath),
|
||||
new HoodieAvroPayload(Option.of((GenericRecord) inMemoryRecord)));
|
||||
|
||||
IndexedRecord onDiskRecord = iRecords.get(99);
|
||||
String dkey = ((GenericRecord) onDiskRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String dPartitionPath = ((GenericRecord) onDiskRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
|
||||
HoodieRecord onDiskHoodieRecord = new HoodieRecord<>(new HoodieKey(dkey, dPartitionPath),
|
||||
new HoodieAvroPayload(Option.of((GenericRecord) onDiskRecord)));
|
||||
// assert size
|
||||
assert records.size() == 100;
|
||||
// get should return the same HoodieKey, same location and same value
|
||||
assert inMemoryHoodieRecord.getKey().equals(records.get(ikey).getKey());
|
||||
assert onDiskHoodieRecord.getKey().equals(records.get(dkey).getKey());
|
||||
// compare the member variables of HoodieRecord not set by the constructor
|
||||
assert records.get(ikey).getCurrentLocation().getFileId().equals(SpillableMapTestUtils.DUMMY_FILE_ID);
|
||||
assert records.get(ikey).getCurrentLocation().getInstantTime().equals(SpillableMapTestUtils.DUMMY_COMMIT_TIME);
|
||||
|
||||
// test contains
|
||||
assertTrue(records.containsKey(ikey));
|
||||
assertTrue(records.containsKey(dkey));
|
||||
|
||||
// test isEmpty
|
||||
assertFalse(records.isEmpty());
|
||||
|
||||
// test containsAll
|
||||
assertTrue(records.keySet().containsAll(recordKeys));
|
||||
|
||||
// remove (from inMemory and onDisk)
|
||||
HoodieRecord removedRecord = records.remove(ikey);
|
||||
assertTrue(removedRecord != null);
|
||||
assertFalse(records.containsKey(ikey));
|
||||
|
||||
removedRecord = records.remove(dkey);
|
||||
assertTrue(removedRecord != null);
|
||||
assertFalse(records.containsKey(dkey));
|
||||
|
||||
// test clear
|
||||
records.clear();
|
||||
assertTrue(records.size() == 0);
|
||||
}
|
||||
|
||||
@Test(expected = IOException.class)
|
||||
public void simpleTestWithException() throws IOException, URISyntaxException {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>(16L, FAILURE_OUTPUT_PATH,
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B
|
||||
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
|
||||
assert (recordKeys.size() == 100);
|
||||
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
|
||||
while (itr.hasNext()) {
|
||||
throw new IOException("Testing failures...");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk() throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH,
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B
|
||||
|
||||
List<String> recordKeys = new ArrayList<>();
|
||||
// Ensure we spill to disk
|
||||
while (records.getDiskBasedMapNumEntries() < 1) {
|
||||
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
|
||||
recordKeys.addAll(SpillableMapTestUtils.upsertRecords(iRecords, records));
|
||||
}
|
||||
|
||||
// Get a record from the in-Memory map
|
||||
String key = recordKeys.get(0);
|
||||
HoodieRecord record = records.get(key);
|
||||
List<IndexedRecord> recordsToUpdate = new ArrayList<>();
|
||||
recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get());
|
||||
|
||||
String newCommitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
List<String> keysToBeUpdated = new ArrayList<>();
|
||||
keysToBeUpdated.add(key);
|
||||
// Update the commitTime for this record
|
||||
List<IndexedRecord> updatedRecords = SchemaTestUtil
|
||||
.updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime);
|
||||
// Upsert this updated record
|
||||
SpillableMapTestUtils.upsertRecords(updatedRecords, records);
|
||||
GenericRecord gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get();
|
||||
// The record returned for this key should have the updated commitTime
|
||||
assert newCommitTime.contentEquals(gRecord.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString());
|
||||
|
||||
// Get a record from the disk based map
|
||||
key = recordKeys.get(recordKeys.size() - 1);
|
||||
record = records.get(key);
|
||||
recordsToUpdate = new ArrayList<>();
|
||||
recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get());
|
||||
|
||||
newCommitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
keysToBeUpdated = new ArrayList<>();
|
||||
keysToBeUpdated.add(key);
|
||||
// Update the commitTime for this record
|
||||
updatedRecords = SchemaTestUtil.updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime);
|
||||
// Upsert this updated record
|
||||
SpillableMapTestUtils.upsertRecords(updatedRecords, records);
|
||||
gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get();
|
||||
// The record returned for this key should have the updated commitTime
|
||||
assert newCommitTime.contentEquals(gRecord.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDataCorrectnessWithoutHoodieMetadata() throws IOException, URISyntaxException {
|
||||
|
||||
Schema schema = SchemaTestUtil.getSimpleSchema();
|
||||
String payloadClazz = HoodieAvroPayload.class.getName();
|
||||
|
||||
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
|
||||
new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH,
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B
|
||||
|
||||
List<String> recordKeys = new ArrayList<>();
|
||||
// Ensure we spill to disk
|
||||
while (records.getDiskBasedMapNumEntries() < 1) {
|
||||
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 100);
|
||||
hoodieRecords.stream().forEach(r -> {
|
||||
records.put(r.getRecordKey(), r);
|
||||
recordKeys.add(r.getRecordKey());
|
||||
});
|
||||
}
|
||||
|
||||
// Get a record from the in-Memory map
|
||||
String key = recordKeys.get(0);
|
||||
HoodieRecord record = records.get(key);
|
||||
// Get the field we want to update
|
||||
String fieldName = schema.getFields().stream().filter(field -> field.schema().getType() == Schema.Type.STRING)
|
||||
.findAny().get().name();
|
||||
// Use a new value to update this field
|
||||
String newValue = "update1";
|
||||
List<HoodieRecord> recordsToUpdate = new ArrayList<>();
|
||||
recordsToUpdate.add(record);
|
||||
|
||||
List<HoodieRecord> updatedRecords =
|
||||
SchemaTestUtil.updateHoodieTestRecordsWithoutHoodieMetadata(recordsToUpdate, schema, fieldName, newValue);
|
||||
|
||||
// Upsert this updated record
|
||||
updatedRecords.stream().forEach(r -> {
|
||||
records.put(r.getRecordKey(), r);
|
||||
});
|
||||
GenericRecord gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get();
|
||||
// The record returned for this key should have the updated value for the field name
|
||||
assertEquals(gRecord.get(fieldName).toString(), newValue);
|
||||
|
||||
// Get a record from the disk based map
|
||||
key = recordKeys.get(recordKeys.size() - 1);
|
||||
record = records.get(key);
|
||||
// Get the field we want to update
|
||||
fieldName = schema.getFields().stream().filter(field -> field.schema().getType() == Schema.Type.STRING)
|
||||
.findAny().get().name();
|
||||
// Use a new value to update this field
|
||||
newValue = "update2";
|
||||
recordsToUpdate = new ArrayList<>();
|
||||
recordsToUpdate.add(record);
|
||||
|
||||
updatedRecords =
|
||||
SchemaTestUtil.updateHoodieTestRecordsWithoutHoodieMetadata(recordsToUpdate, schema, fieldName, newValue);
|
||||
|
||||
// Upsert this updated record
|
||||
updatedRecords.stream().forEach(r -> {
|
||||
records.put(r.getRecordKey(), r);
|
||||
});
|
||||
gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get();
|
||||
// The record returned for this key should have the updated value for the field name
|
||||
assertEquals(gRecord.get(fieldName).toString(), newValue);
|
||||
}
|
||||
|
||||
// TODO : come up with a performance eval test for spillableMap
|
||||
@Test
|
||||
public void testLargeInsertUpsert() {
|
||||
}
|
||||
}
|
||||
@@ -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 org.apache.hudi.common.util.collection;
|
||||
|
||||
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.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.SchemaTestUtil;
|
||||
import org.apache.hudi.common.util.SpillableMapTestUtils;
|
||||
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());
|
||||
}
|
||||
}
|
||||
17
hudi-common/src/test/resources/complex-test-evolved.avsc
Normal file
17
hudi-common/src/test/resources/complex-test-evolved.avsc
Normal file
@@ -0,0 +1,17 @@
|
||||
{
|
||||
"namespace": "example.avro",
|
||||
"type": "record",
|
||||
"name": "User",
|
||||
"fields": [
|
||||
{"name": "field1", "type": ["null", "string"], "default": null},
|
||||
{"name": "field2", "type": ["null", "string"], "default": null},
|
||||
{"name": "name", "type": ["null", "string"], "default": null},
|
||||
{"name": "favoriteIntNumber", "type": ["null", "int"], "default": null},
|
||||
{"name": "favoriteNumber", "type": ["null", "long"], "default": null},
|
||||
{"name": "favoriteFloatNumber", "type": ["null", "float"], "default": null},
|
||||
{"name": "favoriteDoubleNumber", "type": ["null", "double"], "default": null},
|
||||
{"name": "tags", "type": ["null", {"values": ["null", {"fields": [{"default": null, "type": ["null", "string"], "name": "item1"}, {"default": null, "type": ["null", "string"], "name": "item2"} ], "type": "record", "name": "tagsMapItems"} ], "type": "map"} ], "default": null},
|
||||
{"default": null, "name": "testNestedRecord", "type": ["null", {"fields": [{"default": null, "name": "isAdmin", "type": ["null", "boolean"] }, {"default": null, "name": "userId", "type": ["null", "string"] } ], "name": "notes", "type": "record"}]},
|
||||
{"default": null, "name": "stringArray", "type": ["null", {"items": "string", "type": "array"}]}
|
||||
]
|
||||
}
|
||||
@@ -0,0 +1,23 @@
|
||||
###
|
||||
# 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.
|
||||
###
|
||||
log4j.rootLogger=WARN, A1
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.A1=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
|
||||
27
hudi-common/src/test/resources/log4j-surefire.properties
Normal file
27
hudi-common/src/test/resources/log4j-surefire.properties
Normal file
@@ -0,0 +1,27 @@
|
||||
###
|
||||
# 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.
|
||||
###
|
||||
log4j.rootLogger=WARN, A1
|
||||
log4j.category.com.uber=INFO
|
||||
log4j.category.com.uber.hoodie.table.log=WARN
|
||||
log4j.category.com.uber.hoodie.common.util=WARN
|
||||
log4j.category.org.apache.parquet.hadoop=WARN
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.A1=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
|
||||
1000
hudi-common/src/test/resources/sample.data
Normal file
1000
hudi-common/src/test/resources/sample.data
Normal file
File diff suppressed because it is too large
Load Diff
13
hudi-common/src/test/resources/simple-test-evolved.avsc
Normal file
13
hudi-common/src/test/resources/simple-test-evolved.avsc
Normal file
@@ -0,0 +1,13 @@
|
||||
{
|
||||
"namespace": "example.avro",
|
||||
"type": "record",
|
||||
"name": "User",
|
||||
"fields": [
|
||||
{"name": "field1", "type": ["null", "string"], "default": null},
|
||||
{"name": "field2", "type": ["null", "string"], "default": null},
|
||||
{"name": "name", "type": ["null", "string"], "default": null},
|
||||
{"name": "favorite_number", "type": ["null", "long"], "default": null},
|
||||
{"name": "favorite_color", "type": ["null", "string"], "default": null},
|
||||
{"name": "favorite_movie", "type": ["null", "string"], "default": null}
|
||||
]
|
||||
}
|
||||
10
hudi-common/src/test/resources/simple-test.avsc
Normal file
10
hudi-common/src/test/resources/simple-test.avsc
Normal file
@@ -0,0 +1,10 @@
|
||||
{
|
||||
"namespace": "example.avro",
|
||||
"type": "record",
|
||||
"name": "User",
|
||||
"fields": [
|
||||
{"name": "name", "type": "string"},
|
||||
{"name": "favorite_number", "type": "int"},
|
||||
{"name": "favorite_color", "type": "string"}
|
||||
]
|
||||
}
|
||||
Reference in New Issue
Block a user