Skip to content

Commit

Permalink
Add test for PrestoS3InputStream positional read EOS
Browse files Browse the repository at this point in the history
6d296f9 fixed a bug where it returned 0
on EOS but didn't add a test case.
  • Loading branch information
aweisberg authored and rschlussel committed Aug 30, 2019
1 parent d591d79 commit cbe9375
Showing 1 changed file with 36 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,10 @@
import com.amazonaws.services.s3.model.EncryptionMaterials;
import com.amazonaws.services.s3.model.EncryptionMaterialsProvider;
import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
import com.amazonaws.services.s3.model.GetObjectRequest;
import com.amazonaws.services.s3.model.ObjectMetadata;
import com.amazonaws.services.s3.model.S3Object;
import com.amazonaws.services.s3.model.S3ObjectInputStream;
import com.facebook.presto.hive.s3.PrestoS3FileSystem.UnrecoverableS3OperationException;
import com.google.common.base.VerifyException;
import org.apache.hadoop.conf.Configuration;
Expand All @@ -41,12 +44,14 @@

import javax.crypto.spec.SecretKeySpec;

import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.lang.reflect.Field;
import java.net.URI;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;

import static com.facebook.presto.hive.s3.PrestoS3FileSystem.S3_DIRECTORY_OBJECT_CONTENT_TYPE;
import static com.facebook.presto.hive.s3.S3ConfigurationUpdater.S3_ACCESS_KEY;
Expand Down Expand Up @@ -625,4 +630,35 @@ public ObjectMetadata getObjectMetadata(GetObjectMetadataRequest getObjectMetada
assertTrue(fileStatus.isDirectory());
}
}

@Test
public void testPrestoS3InputStreamEOS() throws Exception
{
try (PrestoS3FileSystem fs = new PrestoS3FileSystem()) {
AtomicInteger readableBytes = new AtomicInteger(1);
MockAmazonS3 s3 = new MockAmazonS3()
{
@Override
public S3Object getObject(GetObjectRequest req)
{
return new S3Object()
{
@Override
public S3ObjectInputStream getObjectContent()
{
return new S3ObjectInputStream(new ByteArrayInputStream(new byte[readableBytes.get()]), null);
}
};
}
};
fs.initialize(new URI("s3n://test-bucket/"), new Configuration());
fs.setS3Client(s3);

try (FSDataInputStream inputStream = fs.open(new Path("s3n://test-bucket/test"))) {
assertEquals(inputStream.read(0, new byte[2], 0, 2), 1);
readableBytes.set(0);
assertEquals(inputStream.read(0, new byte[1], 0, 1), -1);
}
}
}
}

0 comments on commit cbe9375

Please sign in to comment.