1
+
1
2
/*
2
3
* Copyright (2021) The Delta Lake Project Authors.
3
4
*
15
16
*/
16
17
17
18
package io .delta .storage ;
18
-
19
+ import io . delta . storage . internal . S3LogStoreUtil ;
19
20
import io .delta .storage .utils .ReflectionUtils ;
20
21
import org .apache .hadoop .fs .Path ;
22
+ import com .google .common .collect .Lists ;
21
23
22
24
import java .io .InterruptedIOException ;
23
25
import java .io .UncheckedIOException ;
26
+ import java .net .URI ;
27
+ import java .util .ArrayList ;
24
28
import java .util .Arrays ;
29
+ import java .util .Collections ;
30
+ import java .util .Comparator ;
31
+ import java .util .HashMap ;
32
+ import java .util .Iterator ;
25
33
import java .util .List ;
26
34
import java .util .Map ;
27
35
import java .util .Optional ;
28
36
import java .util .concurrent .ConcurrentHashMap ;
37
+ import java .util .stream .Collectors ;
38
+ import java .util .stream .Stream ;
29
39
import java .io .IOException ;
40
+ import org .apache .hadoop .fs .FileStatus ;
41
+ import org .apache .hadoop .fs .FileSystem ;
30
42
31
43
import org .apache .hadoop .conf .Configuration ;
32
-
33
44
import com .amazonaws .auth .AWSCredentialsProvider ;
34
45
import com .amazonaws .services .dynamodbv2 .AmazonDynamoDBClient ;
35
46
import com .amazonaws .services .dynamodbv2 .model .AttributeDefinition ;
@@ -87,7 +98,6 @@ public class S3DynamoDBLogStore extends BaseExternalLogStore {
87
98
public static final String DDB_CLIENT_CREDENTIALS_PROVIDER = "credentials.provider" ;
88
99
public static final String DDB_CREATE_TABLE_RCU = "provisionedThroughput.rcu" ;
89
100
public static final String DDB_CREATE_TABLE_WCU = "provisionedThroughput.wcu" ;
90
-
91
101
// WARNING: setting this value too low can cause data loss. Defaults to a duration of 1 day.
92
102
public static final String TTL_SECONDS = "ddb.ttl" ;
93
103
@@ -100,6 +110,8 @@ public class S3DynamoDBLogStore extends BaseExternalLogStore {
100
110
private static final String ATTR_COMPLETE = "complete" ;
101
111
private static final String ATTR_EXPIRE_TIME = "expireTime" ;
102
112
113
+ private static final String ATTR_FILE_SIZE = "fileSize" ;
114
+ private static final String ATTR_MODI_TIME = "modificationTime" ;
103
115
/**
104
116
* Member fields
105
117
*/
@@ -139,6 +151,84 @@ public S3DynamoDBLogStore(Configuration hadoopConf) throws IOException {
139
151
tryEnsureTableExists (hadoopConf );
140
152
}
141
153
154
+ private Iterator <FileStatus > mergeFileLists (
155
+ List <FileStatus > list ,
156
+ List <FileStatus > listWithPrecedence ) {
157
+ final Map <Path , FileStatus > fileStatusMap = new HashMap <>();
158
+
159
+ // insert all elements from `listWithPrecedence` (highest priority)
160
+ // and then insert elements from `list` if and only if that key doesn't already exist
161
+ Stream .concat (listWithPrecedence .stream (), list .stream ())
162
+ .forEach (fs -> fileStatusMap .putIfAbsent (fs .getPath (), fs ));
163
+
164
+ return fileStatusMap
165
+ .values ()
166
+ .stream ()
167
+ .sorted (Comparator .comparing (a -> a .getPath ().getName ()))
168
+ .iterator ();
169
+ }
170
+
171
+ /**
172
+ * List files starting from `resolvedPath` (inclusive) in the same directory.
173
+ */
174
+ private List <FileStatus > listFromCache (
175
+ FileSystem fs ,
176
+ Path resolvedPath ) {
177
+ final Path pathKey = super .stripUserInfo (resolvedPath );
178
+ final Path tablePath = getTablePath (resolvedPath );
179
+
180
+ final Map <String , Condition > conditions = new ConcurrentHashMap <>();
181
+ conditions .put (
182
+ ATTR_TABLE_PATH ,
183
+ new Condition ()
184
+ .withComparisonOperator (ComparisonOperator .EQ )
185
+ .withAttributeValueList (new AttributeValue (tablePath .toString ()))
186
+ );
187
+ conditions .put (ATTR_FILE_NAME , new Condition ().withComparisonOperator (ComparisonOperator .GE )
188
+ .withAttributeValueList (new AttributeValue (pathKey .getName ())));
189
+
190
+ final List <Map <String ,AttributeValue >> items = client .query (
191
+ new QueryRequest (tableName )
192
+ .withConsistentRead (true )
193
+ .withScanIndexForward (false )
194
+ .withLimit (1 )
195
+ .withKeyConditions (conditions )
196
+ ).getItems ();
197
+
198
+ List <FileStatus > statuses = new ArrayList <FileStatus >();
199
+ items .forEach (item -> {
200
+ ExternalCommitEntry entry = dbResultToCommitEntry (item );
201
+ if (entry .complete == true ) {
202
+ long fileSize = entry .fileSize != null ? entry .fileSize : 0L ;
203
+ long modificationTime = entry .modificationTime != null ? entry .modificationTime : System .currentTimeMillis ();
204
+ statuses .add (
205
+ new FileStatus (
206
+ entry .fileSize ,
207
+ false ,
208
+ 1 ,
209
+ fileSize ,
210
+ modificationTime ,
211
+ entry .absoluteFilePath ()
212
+ ));
213
+ }
214
+ });
215
+ return statuses ;
216
+ }
217
+
218
+ @ Override
219
+ public Iterator <FileStatus > listFrom (Path path , Configuration hadoopConf ) throws IOException {
220
+ final FileSystem fs = path .getFileSystem (hadoopConf );
221
+ final Path resolvedPath = stripUserInfo (fs .makeQualified (path ));
222
+
223
+ final List <FileStatus > listedFromFs = Lists .newArrayList (super .listFrom (path , hadoopConf ));
224
+
225
+ // add this to list the completed entry from external cache
226
+ // on the occasion that the filesystem could not provide strong consistency
227
+ final List <FileStatus > listedFromCache = listFromCache (fs , resolvedPath );
228
+
229
+ return mergeFileLists (listedFromCache , listedFromFs );
230
+ }
231
+
142
232
@ Override
143
233
public CloseableIterator <String > read (Path path , Configuration hadoopConf ) throws IOException {
144
234
// With many concurrent readers/writers, there's a chance that concurrent 'recovery'
@@ -222,11 +312,16 @@ protected Optional<ExternalCommitEntry> getLatestExternalEntry(Path tablePath) {
222
312
*/
223
313
private ExternalCommitEntry dbResultToCommitEntry (Map <String , AttributeValue > item ) {
224
314
final AttributeValue expireTimeAttr = item .get (ATTR_EXPIRE_TIME );
315
+ Long fileSize = item .get (ATTR_FILE_SIZE ) != null ? Long .parseLong (item .get (ATTR_FILE_SIZE ).getN ()) : 0 ;
316
+ Long modiTime = item .get (ATTR_MODI_TIME ) != null ? Long .parseLong (item .get (ATTR_MODI_TIME ).getN ()) : System .currentTimeMillis () / 1000 ;
317
+
225
318
return new ExternalCommitEntry (
226
319
new Path (item .get (ATTR_TABLE_PATH ).getS ()),
227
320
item .get (ATTR_FILE_NAME ).getS (),
228
321
item .get (ATTR_TEMP_PATH ).getS (),
229
322
item .get (ATTR_COMPLETE ).getS ().equals ("true" ),
323
+ fileSize ,
324
+ modiTime ,
230
325
expireTimeAttr != null ? Long .parseLong (expireTimeAttr .getN ()) : null
231
326
);
232
327
}
@@ -236,6 +331,11 @@ private PutItemRequest createPutItemRequest(ExternalCommitEntry entry, boolean o
236
331
attributes .put (ATTR_TABLE_PATH , new AttributeValue (entry .tablePath .toString ()));
237
332
attributes .put (ATTR_FILE_NAME , new AttributeValue (entry .fileName ));
238
333
attributes .put (ATTR_TEMP_PATH , new AttributeValue (entry .tempPath ));
334
+ attributes .put (ATTR_FILE_SIZE , new AttributeValue ().withN (
335
+ String .valueOf (entry .fileSize != null ? entry .fileSize : 0L )));
336
+ attributes .put (ATTR_MODI_TIME , new AttributeValue ().withN (
337
+ String .valueOf (entry .modificationTime != null ? entry .modificationTime : System .currentTimeMillis ())
338
+ ));
239
339
attributes .put (
240
340
ATTR_COMPLETE ,
241
341
new AttributeValue ().withS (Boolean .toString (entry .complete ))
0 commit comments