mirror of https://github.com/apache/druid.git
support atomic writes for local deep storage (#3521)
* Use atomic writes for local deep storage * fix pr issues * use defaultObjMapper for test * move tmp pushes to a intermediate dir * minor refactor
This commit is contained in:
parent
4be3eb0ce7
commit
48b22e261a
|
@ -20,6 +20,7 @@
|
||||||
package io.druid.timeline.partition;
|
package io.druid.timeline.partition;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Range;
|
import com.google.common.collect.Range;
|
||||||
import io.druid.data.input.InputRow;
|
import io.druid.data.input.InputRow;
|
||||||
|
@ -55,6 +56,7 @@ public class NoneShardSpec implements ShardSpec
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@JsonIgnore
|
||||||
public int getPartitionNum()
|
public int getPartitionNum()
|
||||||
{
|
{
|
||||||
return 0;
|
return 0;
|
||||||
|
|
|
@ -5,6 +5,8 @@ import io.druid.TestObjectMapper;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
public class NoneShardSpecTest
|
public class NoneShardSpecTest
|
||||||
{
|
{
|
||||||
@Test
|
@Test
|
||||||
|
@ -28,4 +30,13 @@ public class NoneShardSpecTest
|
||||||
Assert.assertTrue(serde1 == serde2);
|
Assert.assertTrue(serde1 == serde2);
|
||||||
Assert.assertTrue(one == serde1);
|
Assert.assertTrue(one == serde1);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testPartitionFieldIgnored() throws IOException
|
||||||
|
{
|
||||||
|
final String jsonStr = "{\"type\": \"none\",\"partitionNum\": 2}";
|
||||||
|
ObjectMapper mapper = new TestObjectMapper();
|
||||||
|
final ShardSpec noneShardSpec = mapper.readValue(jsonStr, ShardSpec.class);
|
||||||
|
noneShardSpec.equals(NoneShardSpec.instance());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,9 +29,12 @@ import io.druid.java.util.common.CompressionUtils;
|
||||||
import io.druid.java.util.common.logger.Logger;
|
import io.druid.java.util.common.logger.Logger;
|
||||||
import io.druid.segment.SegmentUtils;
|
import io.druid.segment.SegmentUtils;
|
||||||
import io.druid.timeline.DataSegment;
|
import io.druid.timeline.DataSegment;
|
||||||
|
import org.apache.commons.io.FileUtils;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.nio.file.FileAlreadyExistsException;
|
||||||
|
import java.util.UUID;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -70,7 +73,9 @@ public class LocalDataSegmentPusher implements DataSegmentPusher
|
||||||
@Override
|
@Override
|
||||||
public DataSegment push(File dataSegmentFile, DataSegment segment) throws IOException
|
public DataSegment push(File dataSegmentFile, DataSegment segment) throws IOException
|
||||||
{
|
{
|
||||||
File outDir = new File(config.getStorageDirectory(), DataSegmentPusherUtil.getStorageDir(segment));
|
final String storageDir = DataSegmentPusherUtil.getStorageDir(segment);
|
||||||
|
final File baseStorageDir = config.getStorageDirectory();
|
||||||
|
final File outDir = new File(baseStorageDir, storageDir);
|
||||||
|
|
||||||
log.info("Copying segment[%s] to local filesystem at location[%s]", segment.getIdentifier(), outDir.toString());
|
log.info("Copying segment[%s] to local filesystem at location[%s]", segment.getIdentifier(), outDir.toString());
|
||||||
|
|
||||||
|
@ -88,19 +93,49 @@ public class LocalDataSegmentPusher implements DataSegmentPusher
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!outDir.mkdirs() && !outDir.isDirectory()) {
|
final File tmpOutDir = new File(baseStorageDir, intermediateDirFor(storageDir));
|
||||||
throw new IOException(String.format("Cannot create directory[%s]", outDir));
|
log.info("Creating intermediate directory[%s] for segment[%s]", tmpOutDir.toString(), segment.getIdentifier());
|
||||||
}
|
final long size = compressSegment(dataSegmentFile, tmpOutDir);
|
||||||
File outFile = new File(outDir, "index.zip");
|
|
||||||
log.info("Compressing files from[%s] to [%s]", dataSegmentFile, outFile);
|
|
||||||
long size = CompressionUtils.zip(dataSegmentFile, outFile);
|
|
||||||
|
|
||||||
return createDescriptorFile(
|
final DataSegment dataSegment = createDescriptorFile(
|
||||||
segment.withLoadSpec(makeLoadSpec(outFile))
|
segment.withLoadSpec(makeLoadSpec(new File(outDir, "index.zip")))
|
||||||
.withSize(size)
|
.withSize(size)
|
||||||
.withBinaryVersion(SegmentUtils.getVersionFromDir(dataSegmentFile)),
|
.withBinaryVersion(SegmentUtils.getVersionFromDir(dataSegmentFile)),
|
||||||
outDir
|
tmpOutDir
|
||||||
);
|
);
|
||||||
|
|
||||||
|
// moving the temporary directory to the final destination, once success the potentially concurrent push operations
|
||||||
|
// will be failed and will read the descriptor.json created by current push operation directly
|
||||||
|
createDirectoryIfNotExists(outDir.getParentFile());
|
||||||
|
try {
|
||||||
|
java.nio.file.Files.move(tmpOutDir.toPath(), outDir.toPath());
|
||||||
|
}
|
||||||
|
catch (FileAlreadyExistsException e) {
|
||||||
|
log.warn("Push destination directory[%s] exists, ignore this message if replication is configured.", outDir);
|
||||||
|
FileUtils.deleteDirectory(tmpOutDir);
|
||||||
|
return jsonMapper.readValue(new File(outDir, "descriptor.json"), DataSegment.class);
|
||||||
|
}
|
||||||
|
return dataSegment;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void createDirectoryIfNotExists(File directory) throws IOException
|
||||||
|
{
|
||||||
|
if (!directory.mkdirs() && !directory.isDirectory()) {
|
||||||
|
throw new IOException(String.format("Cannot create directory[%s]", directory.toString()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private String intermediateDirFor(String storageDir)
|
||||||
|
{
|
||||||
|
return "intermediate_pushes/" + storageDir + "." + UUID.randomUUID().toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
private long compressSegment(File dataSegmentFile, File outDir) throws IOException
|
||||||
|
{
|
||||||
|
createDirectoryIfNotExists(outDir);
|
||||||
|
File outFile = new File(outDir, "index.zip");
|
||||||
|
log.info("Compressing files from[%s] to [%s]", dataSegmentFile, outFile);
|
||||||
|
return CompressionUtils.zip(dataSegmentFile, outFile);
|
||||||
}
|
}
|
||||||
|
|
||||||
private DataSegment createDescriptorFile(DataSegment segment, File outDir) throws IOException
|
private DataSegment createDescriptorFile(DataSegment segment, File outDir) throws IOException
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
|
import io.druid.jackson.DefaultObjectMapper;
|
||||||
import io.druid.timeline.DataSegment;
|
import io.druid.timeline.DataSegment;
|
||||||
import io.druid.timeline.partition.NoneShardSpec;
|
import io.druid.timeline.partition.NoneShardSpec;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
@ -64,7 +65,7 @@ public class LocalDataSegmentPusherTest
|
||||||
{
|
{
|
||||||
config = new LocalDataSegmentPusherConfig();
|
config = new LocalDataSegmentPusherConfig();
|
||||||
config.storageDirectory = temporaryFolder.newFolder();
|
config.storageDirectory = temporaryFolder.newFolder();
|
||||||
localDataSegmentPusher = new LocalDataSegmentPusher(config, new ObjectMapper());
|
localDataSegmentPusher = new LocalDataSegmentPusher(config, new DefaultObjectMapper());
|
||||||
dataSegmentFiles = temporaryFolder.newFolder();
|
dataSegmentFiles = temporaryFolder.newFolder();
|
||||||
Files.asByteSink(new File(dataSegmentFiles, "version.bin")).write(Ints.toByteArray(0x9));
|
Files.asByteSink(new File(dataSegmentFiles, "version.bin")).write(Ints.toByteArray(0x9));
|
||||||
}
|
}
|
||||||
|
@ -103,6 +104,17 @@ public class LocalDataSegmentPusherTest
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFirstPushWinsForConcurrentPushes() throws IOException
|
||||||
|
{
|
||||||
|
File replicatedDataSegmentFiles = temporaryFolder.newFolder();
|
||||||
|
Files.asByteSink(new File(replicatedDataSegmentFiles, "version.bin")).write(Ints.toByteArray(0x8));
|
||||||
|
DataSegment returnSegment1 = localDataSegmentPusher.push(dataSegmentFiles, dataSegment);
|
||||||
|
DataSegment returnSegment2 = localDataSegmentPusher.push(replicatedDataSegmentFiles, dataSegment);
|
||||||
|
|
||||||
|
Assert.assertEquals(returnSegment1, returnSegment2);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testPushCannotCreateDirectory() throws IOException
|
public void testPushCannotCreateDirectory() throws IOException
|
||||||
{
|
{
|
||||||
|
|
Loading…
Reference in New Issue