Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Ignore empty folder placeholder files on S3 #4552

Merged
merged 1 commit into from
Jul 27, 2020
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 @@ -16,10 +16,15 @@
import com.google.common.collect.ImmutableSet;
import io.prestosql.plugin.hive.s3.HiveS3Config;
import io.prestosql.plugin.hive.s3.PrestoS3ConfigurationInitializer;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.testng.annotations.Test;

import java.util.Arrays;

import static com.google.common.base.Preconditions.checkArgument;
import static java.lang.String.format;
import static org.testng.Assert.assertFalse;
import static org.testng.util.Strings.isNullOrEmpty;

public abstract class AbstractTestHiveFileSystemS3
Expand Down Expand Up @@ -69,4 +74,18 @@ protected Path getBasePath()
// HDP 3.1 does not understand s3:// out of the box.
return new Path(format("s3a://%s/%s/", writableBucket, testDirectory));
}

@Test
public void testIgnoreHadoopFolderMarker()
throws Exception
{
Path basePath = getBasePath();
FileSystem fs = hdfsEnvironment.getFileSystem(TESTING_CONTEXT, basePath);

String markerFileName = "test_table_$folder$";
Path filePath = new Path(basePath, markerFileName);
fs.create(filePath).close();

assertFalse(Arrays.stream(fs.listStatus(basePath)).anyMatch(file -> file.getPath().getName().equalsIgnoreCase(markerFileName)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -575,6 +575,7 @@ private Iterator<LocatedFileStatus> statusFromObjects(List<S3ObjectSummary> obje
return objects.stream()
.filter(object -> !object.getKey().endsWith(PATH_SEPARATOR))
.filter(object -> !skipGlacierObjects || !isGlacierObject(object))
.filter(object -> !isHadoopFolderMarker(object))
.map(object -> new FileStatus(
object.getSize(),
false,
Expand All @@ -591,6 +592,11 @@ private static boolean isGlacierObject(S3ObjectSummary object)
return Glacier.toString().equals(object.getStorageClass());
}

private static boolean isHadoopFolderMarker(S3ObjectSummary object)
{
return object.getKey().endsWith("_$folder$");
}

/**
* This exception is for stopping retries for S3 calls that shouldn't be retried.
* For example, "Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Forbidden (Service: Amazon S3; Status Code: 403 ..."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ public class MockAmazonS3
private GetObjectMetadataRequest getObjectMetadataRequest;
private CannedAccessControlList acl;
private boolean hasGlacierObjects;
private boolean hasHadoopFolderMarkerObjects;

public void setGetObjectHttpErrorCode(int getObjectHttpErrorCode)
{
Expand All @@ -60,6 +61,11 @@ public void setHasGlacierObjects(boolean hasGlacierObjects)
this.hasGlacierObjects = hasGlacierObjects;
}

public void setHasHadoopFolderMarkerObjects(boolean hasHadoopFolderMarkerObjects)
{
this.hasHadoopFolderMarkerObjects = hasHadoopFolderMarkerObjects;
}

public GetObjectMetadataRequest getGetObjectMetadataRequest()
{
return getObjectMetadataRequest;
Expand Down Expand Up @@ -125,6 +131,14 @@ public ListObjectsV2Result listObjectsV2(ListObjectsV2Request listObjectsV2Reque
listingV2.getObjectSummaries().add(standardOne);
listingV2.setTruncated(true);
listingV2.setNextContinuationToken(continuationToken);

if (hasHadoopFolderMarkerObjects) {
S3ObjectSummary hadoopFolderMarker = new S3ObjectSummary();
hadoopFolderMarker.setStorageClass(StorageClass.Standard.toString());
hadoopFolderMarker.setKey("test/test_$folder$");
hadoopFolderMarker.setLastModified(new Date());
listingV2.getObjectSummaries().add(hadoopFolderMarker);
}
}

return listingV2;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -570,6 +570,22 @@ private static void assertSkipGlacierObjects(boolean skipGlacierObjects)
}
}

@Test
public void testSkipHadoopFolderMarkerObjectsEnabled()
throws Exception
{
Configuration config = new Configuration(false);

try (PrestoS3FileSystem fs = new PrestoS3FileSystem()) {
MockAmazonS3 s3 = new MockAmazonS3();
s3.setHasHadoopFolderMarkerObjects(true);
fs.initialize(new URI("s3n://test-bucket/"), config);
fs.setS3Client(s3);
FileStatus[] statuses = fs.listStatus(new Path("s3n://test-bucket/test"));
assertEquals(statuses.length, 2);
}
}

public static AWSCredentialsProvider getAwsCredentialsProvider(PrestoS3FileSystem fs)
{
return getFieldValue(fs.getS3Client(), "awsCredentialsProvider", AWSCredentialsProvider.class);
Expand Down