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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -280,22 +280,22 @@ public static long getContainerID(File containerBaseDir) {
return Long.parseLong(containerBaseDir.getName());
}

public static String getContainerTarGzName(long containerId) {
return "container-" + containerId + ".tar.gz";
public static String getContainerTarName(long containerId) {
return "container-" + containerId + ".tar";
}

public static long retrieveContainerIdFromTarGzName(String tarGzName)
public static long retrieveContainerIdFromTarName(String tarName)
throws IOException {
assert tarGzName != null;
Pattern pattern = Pattern.compile("container-(\\d+).tar.gz");
assert tarName != null;
Pattern pattern = Pattern.compile("container-(\\d+).tar");
// Now create matcher object.
Matcher m = pattern.matcher(tarGzName);
Matcher m = pattern.matcher(tarName);

if (m.find()) {
return Long.parseLong(m.group(1));
} else {
throw new IOException("Illegal container tar gz file " +
tarGzName);
tarName);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@
import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_V3;

/**
* Compress/uncompress KeyValueContainer data to a tar.gz archive.
* Compress/uncompress KeyValueContainer data to a tar archive.
*/
public class TarContainerPacker
implements ContainerPacker<KeyValueContainerData> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ public CompletableFuture<Path> download(long containerId) {
CompletableFuture<Path> response = new CompletableFuture<>();

Path destinationPath = getWorkingDirectory()
.resolve(ContainerUtils.getContainerTarGzName(containerId));
.resolve(ContainerUtils.getContainerTarName(containerId));

client.download(request,
new StreamDownloader(containerId, response, destinationPath));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,12 +59,12 @@ public void redactsDataBuffers() {
}

@Test
public void testTarGzName() throws IOException {
public void testTarName() throws IOException {
long containerId = 100;
String tarGzName = "container-100.tar.gz";
assertEquals(tarGzName, ContainerUtils.getContainerTarGzName(containerId));
String tarName = "container-100.tar";
assertEquals(tarName, ContainerUtils.getContainerTarName(containerId));

assertEquals(containerId,
ContainerUtils.retrieveContainerIdFromTarGzName(tarGzName));
ContainerUtils.retrieveContainerIdFromTarName(tarName));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -193,7 +193,7 @@ public void testEmptyContainerImportExport() throws Exception {
checkContainerFilesPresent(data, 0);

//destination path
File exportTar = folder.newFile("exported.tar.gz");
File exportTar = folder.newFile("exported.tar");
TarContainerPacker packer = new TarContainerPacker();
//export the container
try (FileOutputStream fos = new FileOutputStream(exportTar)) {
Expand All @@ -220,7 +220,7 @@ public void testContainerImportExport() throws Exception {
populate(numberOfKeysToWrite);

//destination path
File folderToExport = folder.newFile("exported.tar.gz");
File folderToExport = folder.newFile("exported.tar");
for (Map.Entry<CopyContainerCompression, String> entry :
CopyContainerCompression.getCompressionMapping().entrySet()) {
TarContainerPacker packer = new TarContainerPacker(entry.getValue());
Expand Down Expand Up @@ -368,7 +368,7 @@ public void concurrentExport() throws Exception {
List<Thread> threads = IntStream.range(0, 20)
.mapToObj(i -> new Thread(() -> {
try {
File file = folder.newFile("concurrent" + i + ".tar.gz");
File file = folder.newFile("concurrent" + i + ".tar");
try (OutputStream out = new FileOutputStream(file)) {
keyValueContainer.exportContainerData(out, packer);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -182,7 +182,7 @@ public void pack() throws IOException, CompressorException {
//sample container descriptor file
writeDescriptor(sourceContainer);

Path targetFile = TEMP_DIR.resolve("container.tar.gz");
Path targetFile = TEMP_DIR.resolve("container.tar");

//WHEN: pack it
try (FileOutputStream output = new FileOutputStream(targetFile.toFile())) {
Expand Down Expand Up @@ -372,7 +372,7 @@ private File writeSingleFile(Path parentPath, String fileName,

private File packContainerWithSingleFile(File file, String entryName)
throws Exception {
File targetFile = TEMP_DIR.resolve("container.tar.gz").toFile();
File targetFile = TEMP_DIR.resolve("container.tar").toFile();
try (FileOutputStream output = new FileOutputStream(targetFile);
OutputStream compressed = packer.compress(output);
ArchiveOutputStream archive = new TarArchiveOutputStream(compressed)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -674,7 +674,7 @@ public void importContainer(long containerID, File source) throws Exception {
new TarContainerPacker(), dsm.getContainer().getVolumeSet());

File tempFile = tempFolder.newFile(
ContainerUtils.getContainerTarGzName(containerID));
ContainerUtils.getContainerTarName(containerID));
Files.copy(source.toPath(), tempFile.toPath(),
StandardCopyOption.REPLACE_EXISTING);
replicator.importContainer(containerID, tempFile.toPath(), null);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,39 +21,27 @@
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.file.Files;
import java.nio.file.Path;
import java.time.Duration;
import java.time.Instant;
import java.util.Collection;
import java.util.stream.Collectors;
import java.util.stream.Stream;

import org.apache.hadoop.hdds.server.OzoneAdmins;
import org.apache.hadoop.hdds.utils.db.DBCheckpoint;
import org.apache.hadoop.hdds.utils.db.DBStore;

import org.apache.commons.compress.archivers.ArchiveEntry;
import org.apache.commons.compress.archivers.ArchiveOutputStream;
import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
import org.apache.commons.compress.compressors.CompressorException;
import org.apache.commons.compress.compressors.CompressorOutputStream;
import org.apache.commons.compress.compressors.CompressorStreamFactory;
import org.apache.commons.compress.utils.IOUtils;
import org.apache.commons.lang3.StringUtils;

import static org.apache.hadoop.hdds.utils.HddsServerUtil.writeDBCheckpointToStream;
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_REQUEST_FLUSH;

import org.apache.hadoop.security.UserGroupInformation;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* Provides the current checkpoint Snapshot of the OM/SCM DB. (tar.gz)
* Provides the current checkpoint Snapshot of the OM/SCM DB. (tar)
*/
public class DBCheckpointServlet extends HttpServlet {

Expand Down Expand Up @@ -168,10 +156,10 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) {
if (file == null) {
return;
}
response.setContentType("application/x-tgz");
response.setContentType("application/x-tar");
response.setHeader("Content-Disposition",
"attachment; filename=\"" +
file.toString() + ".tgz\"");
file + ".tar\"");

Instant start = Instant.now();
writeDBCheckpointToStream(checkpoint,
Expand Down Expand Up @@ -200,53 +188,4 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) {
}
}

/**
* Write DB Checkpoint to an output stream as a compressed file (tgz).
*
* @param checkpoint checkpoint file
* @param destination desination output stream.
* @throws IOException
*/
public static void writeDBCheckpointToStream(DBCheckpoint checkpoint,
OutputStream destination)
throws IOException {

try (CompressorOutputStream gzippedOut = new CompressorStreamFactory()
.createCompressorOutputStream(CompressorStreamFactory.GZIP,
destination)) {

try (ArchiveOutputStream archiveOutputStream =
new TarArchiveOutputStream(gzippedOut)) {

Path checkpointPath = checkpoint.getCheckpointLocation();
try (Stream<Path> files = Files.list(checkpointPath)) {
for (Path path : files.collect(Collectors.toList())) {
if (path != null) {
Path fileName = path.getFileName();
if (fileName != null) {
includeFile(path.toFile(), fileName.toString(),
archiveOutputStream);
}
}
}
}
}
} catch (CompressorException e) {
throw new IOException(
"Can't compress the checkpoint: " +
checkpoint.getCheckpointLocation(), e);
}
}

private static void includeFile(File file, String entryName,
ArchiveOutputStream archiveOutputStream)
throws IOException {
ArchiveEntry archiveEntry =
archiveOutputStream.createArchiveEntry(file, entryName);
archiveOutputStream.putArchiveEntry(archiveEntry);
try (FileInputStream fis = new FileInputStream(file)) {
IOUtils.copy(fis, archiveOutputStream);
}
archiveOutputStream.closeArchiveEntry();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,9 +37,6 @@
import org.apache.commons.compress.archivers.ArchiveEntry;
import org.apache.commons.compress.archivers.ArchiveOutputStream;
import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
import org.apache.commons.compress.compressors.CompressorException;
import org.apache.commons.compress.compressors.CompressorOutputStream;
import org.apache.commons.compress.compressors.CompressorStreamFactory;
import org.apache.commons.compress.utils.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.DFSConfigKeysLegacy;
Expand Down Expand Up @@ -526,7 +523,7 @@ public static MetricsSystem initializeMetrics(
}

/**
* Write DB Checkpoint to an output stream as a compressed file (tgz).
* Write DB Checkpoint to an output stream as a compressed file (tar).
*
* @param checkpoint checkpoint file
* @param destination destination output stream.
Expand All @@ -535,11 +532,8 @@ public static MetricsSystem initializeMetrics(
public static void writeDBCheckpointToStream(DBCheckpoint checkpoint,
OutputStream destination)
throws IOException {
try (CompressorOutputStream gzippedOut = new CompressorStreamFactory()
.createCompressorOutputStream(CompressorStreamFactory.GZIP,
destination);
ArchiveOutputStream archiveOutputStream =
new TarArchiveOutputStream(gzippedOut);
try (ArchiveOutputStream archiveOutputStream =
new TarArchiveOutputStream(destination);
Stream<Path> files =
Files.list(checkpoint.getCheckpointLocation())) {
for (Path path : files.collect(Collectors.toList())) {
Expand All @@ -551,15 +545,11 @@ public static void writeDBCheckpointToStream(DBCheckpoint checkpoint,
}
}
}
} catch (CompressorException e) {
throw new IOException(
"Can't compress the checkpoint: " +
checkpoint.getCheckpointLocation(), e);
}
}

private static void includeFile(File file, String entryName,
ArchiveOutputStream archiveOutputStream)
ArchiveOutputStream archiveOutputStream)
throws IOException {
ArchiveEntry archiveEntry =
archiveOutputStream.createArchiveEntry(file, entryName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ public DBCheckpoint getSCMDBSnapshot(String leaderSCMNodeID)
String snapshotFilePath =
Paths.get(scmSnapshotDir.getAbsolutePath(), snapshotFileName).toFile()
.getAbsolutePath();
File targetFile = new File(snapshotFilePath + ".tar.gz");
File targetFile = new File(snapshotFilePath + ".tar");


// the downloadClient instance will be created as and when install snapshot
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,7 @@ public void testDoGet() throws ServletException, IOException {
Matchers.anyString());

tempFile = File.createTempFile("testDoGet_" + System
.currentTimeMillis(), ".tar.gz");
.currentTimeMillis(), ".tar");

FileOutputStream fileOutputStream = new FileOutputStream(tempFile);
when(responseMock.getOutputStream()).thenReturn(
Expand Down
Loading