2828import java .io .IOException ;
2929import java .io .InputStream ;
3030import java .net .URI ;
31+ import java .util .ArrayList ;
32+ import java .util .Collections ;
33+ import java .util .List ;
3134
3235import org .apache .commons .logging .Log ;
3336import org .apache .commons .logging .LogFactory ;
4144import org .jets3t .service .ServiceException ;
4245import org .jets3t .service .StorageObjectsChunk ;
4346import org .jets3t .service .impl .rest .httpclient .RestS3Service ;
47+ import org .jets3t .service .model .MultipartPart ;
48+ import org .jets3t .service .model .MultipartUpload ;
4449import org .jets3t .service .model .S3Bucket ;
4550import org .jets3t .service .model .S3Object ;
4651import org .jets3t .service .model .StorageObject ;
4752import org .jets3t .service .security .AWSCredentials ;
53+ import org .jets3t .service .utils .MultipartUtils ;
4854
4955@ InterfaceAudience .Private
5056@ InterfaceStability .Unstable
5157class Jets3tNativeFileSystemStore implements NativeFileSystemStore {
5258
5359 private S3Service s3Service ;
5460 private S3Bucket bucket ;
61+
62+ private long multipartBlockSize ;
63+ private boolean multipartEnabled ;
64+ private long multipartCopyBlockSize ;
65+ static final long MAX_PART_SIZE = (long )5 * 1024 * 1024 * 1024 ;
66+
5567 public static final Log LOG =
5668 LogFactory .getLog (Jets3tNativeFileSystemStore .class );
5769
@@ -67,13 +79,27 @@ public void initialize(URI uri, Configuration conf) throws IOException {
6779 } catch (S3ServiceException e ) {
6880 handleS3ServiceException (e );
6981 }
82+ multipartEnabled =
83+ conf .getBoolean ("fs.s3n.multipart.uploads.enabled" , false );
84+ multipartBlockSize = Math .min (
85+ conf .getLong ("fs.s3n.multipart.uploads.block.size" , 64 * 1024 * 1024 ),
86+ MAX_PART_SIZE );
87+ multipartCopyBlockSize = Math .min (
88+ conf .getLong ("fs.s3n.multipart.copy.block.size" , MAX_PART_SIZE ),
89+ MAX_PART_SIZE );
90+
7091 bucket = new S3Bucket (uri .getHost ());
7192 }
7293
7394 @ Override
7495 public void storeFile (String key , File file , byte [] md5Hash )
7596 throws IOException {
76-
97+
98+ if (multipartEnabled && file .length () >= multipartBlockSize ) {
99+ storeLargeFile (key , file , md5Hash );
100+ return ;
101+ }
102+
77103 BufferedInputStream in = null ;
78104 try {
79105 in = new BufferedInputStream (new FileInputStream (file ));
@@ -98,6 +124,31 @@ public void storeFile(String key, File file, byte[] md5Hash)
98124 }
99125 }
100126
127+ public void storeLargeFile (String key , File file , byte [] md5Hash )
128+ throws IOException {
129+ S3Object object = new S3Object (key );
130+ object .setDataInputFile (file );
131+ object .setContentType ("binary/octet-stream" );
132+ object .setContentLength (file .length ());
133+ if (md5Hash != null ) {
134+ object .setMd5Hash (md5Hash );
135+ }
136+
137+ List <StorageObject > objectsToUploadAsMultipart =
138+ new ArrayList <StorageObject >();
139+ objectsToUploadAsMultipart .add (object );
140+ MultipartUtils mpUtils = new MultipartUtils (multipartBlockSize );
141+
142+ try {
143+ mpUtils .uploadObjects (bucket .getName (), s3Service ,
144+ objectsToUploadAsMultipart , null );
145+ } catch (ServiceException e ) {
146+ handleServiceException (e );
147+ } catch (Exception e ) {
148+ throw new S3Exception (e );
149+ }
150+ }
151+
101152 @ Override
102153 public void storeEmptyFile (String key ) throws IOException {
103154 try {
@@ -152,11 +203,8 @@ public InputStream retrieve(String key) throws IOException {
152203 }
153204 S3Object object = s3Service .getObject (bucket .getName (), key );
154205 return object .getDataInputStream ();
155- } catch (S3ServiceException e ) {
156- handleS3ServiceException (key , e );
157- return null ; //never returned - keep compiler happy
158206 } catch (ServiceException e ) {
159- handleServiceException (e );
207+ handleServiceException (key , e );
160208 return null ; //return null if key not found
161209 }
162210 }
@@ -180,11 +228,8 @@ public InputStream retrieve(String key, long byteRangeStart)
180228 S3Object object = s3Service .getObject (bucket , key , null , null , null ,
181229 null , byteRangeStart , null );
182230 return object .getDataInputStream ();
183- } catch (S3ServiceException e ) {
184- handleS3ServiceException (key , e );
185- return null ; //never returned - keep compiler happy
186231 } catch (ServiceException e ) {
187- handleServiceException (e );
232+ handleServiceException (key , e );
188233 return null ; //return null if key not found
189234 }
190235 }
@@ -244,8 +289,16 @@ public void delete(String key) throws IOException {
244289 LOG .debug ("Deleting key:" + key + "from bucket" + bucket .getName ());
245290 }
246291 s3Service .deleteObject (bucket , key );
247- } catch (S3ServiceException e ) {
248- handleS3ServiceException (key , e );
292+ } catch (ServiceException e ) {
293+ handleServiceException (key , e );
294+ }
295+ }
296+
297+ public void rename (String srcKey , String dstKey ) throws IOException {
298+ try {
299+ s3Service .renameObject (bucket .getName (), srcKey , new S3Object (dstKey ));
300+ } catch (ServiceException e ) {
301+ handleServiceException (e );
249302 }
250303 }
251304
@@ -255,10 +308,52 @@ public void copy(String srcKey, String dstKey) throws IOException {
255308 if (LOG .isDebugEnabled ()) {
256309 LOG .debug ("Copying srcKey: " + srcKey + "to dstKey: " + dstKey + "in bucket: " + bucket .getName ());
257310 }
311+ if (multipartEnabled ) {
312+ S3Object object = s3Service .getObjectDetails (bucket , srcKey , null ,
313+ null , null , null );
314+ if (multipartCopyBlockSize > 0 &&
315+ object .getContentLength () > multipartCopyBlockSize ) {
316+ copyLargeFile (object , dstKey );
317+ return ;
318+ }
319+ }
258320 s3Service .copyObject (bucket .getName (), srcKey , bucket .getName (),
259321 new S3Object (dstKey ), false );
260- } catch (S3ServiceException e ) {
261- handleS3ServiceException (srcKey , e );
322+ } catch (ServiceException e ) {
323+ handleServiceException (srcKey , e );
324+ }
325+ }
326+
327+ public void copyLargeFile (S3Object srcObject , String dstKey ) throws IOException {
328+ try {
329+ long partCount = srcObject .getContentLength () / multipartCopyBlockSize +
330+ (srcObject .getContentLength () % multipartCopyBlockSize > 0 ? 1 : 0 );
331+
332+ MultipartUpload multipartUpload = s3Service .multipartStartUpload
333+ (bucket .getName (), dstKey , srcObject .getMetadataMap ());
334+
335+ List <MultipartPart > listedParts = new ArrayList <MultipartPart >();
336+ for (int i = 0 ; i < partCount ; i ++) {
337+ long byteRangeStart = i * multipartCopyBlockSize ;
338+ long byteLength ;
339+ if (i < partCount - 1 ) {
340+ byteLength = multipartCopyBlockSize ;
341+ } else {
342+ byteLength = srcObject .getContentLength () % multipartCopyBlockSize ;
343+ if (byteLength == 0 ) {
344+ byteLength = multipartCopyBlockSize ;
345+ }
346+ }
347+
348+ MultipartPart copiedPart = s3Service .multipartUploadPartCopy
349+ (multipartUpload , i + 1 , bucket .getName (), srcObject .getKey (),
350+ null , null , null , null , byteRangeStart ,
351+ byteRangeStart + byteLength - 1 , null );
352+ listedParts .add (copiedPart );
353+ }
354+
355+ Collections .reverse (listedParts );
356+ s3Service .multipartCompleteUpload (multipartUpload , listedParts );
262357 } catch (ServiceException e ) {
263358 handleServiceException (e );
264359 }
@@ -291,11 +386,11 @@ public void dump() throws IOException {
291386 System .out .println (sb );
292387 }
293388
294- private void handleS3ServiceException (String key , S3ServiceException e ) throws IOException {
295- if ("NoSuchKey" .equals (e .getS3ErrorCode ())) {
389+ private void handleServiceException (String key , ServiceException e ) throws IOException {
390+ if ("NoSuchKey" .equals (e .getErrorCode ())) {
296391 throw new FileNotFoundException ("Key '" + key + "' does not exist in S3" );
297392 } else {
298- handleS3ServiceException (e );
393+ handleServiceException (e );
299394 }
300395 }
301396
0 commit comments