@@ -101,12 +101,45 @@ public class TestDeltaLakeSplitManager
101101 0 );
102102 private final HiveTransactionHandle transactionHandle = new HiveTransactionHandle (true );
103103
104+ @ Test
105+ public void testAbsolutePathSplits ()
106+ throws Exception
107+ {
108+ testAbsolutePathSplits ("file://path/to/file" , "file://path/to/file" );
109+ testAbsolutePathSplits (
"abfs://[email protected] /path/to/file" ,
"abfs://[email protected] /path/to/file" );
110+ testAbsolutePathSplits ("hdfs://path/to/file" , "hdfs://path/to/file" );
111+ testAbsolutePathSplits ("s3://my-s3-bucket/path/to//file" , "s3://my-s3-bucket/path/to//file" );
112+ testAbsolutePathSplits ("s3://my-s3-bucket/path/to//file/" , "s3://my-s3-bucket/path/to//file/" );
113+ testAbsolutePathSplits ("gs://my-gcp-bucket/path/to/file" , "gs://my-gcp-bucket/path/to/file" );
114+ testAbsolutePathSplits (
"abfs://[email protected] /+ab+/a%25/a%2525/path/to/file" ,
"abfs://[email protected] /+ab+/a%/a%25/path/to/file" );
115+ }
116+
117+ private void testAbsolutePathSplits (String absoluteRawEncodedFilePath , String absoluteDecodedParsedFilePath )
118+ throws Exception
119+ {
120+ long fileSize = 20_000 ;
121+ List <AddFileEntry > addFileEntries = ImmutableList .of (addFileEntryOfSize (absoluteRawEncodedFilePath , fileSize ));
122+ DeltaLakeConfig deltaLakeConfig = new DeltaLakeConfig ()
123+ .setMaxSplitSize (DataSize .ofBytes (5_000 ));
124+ double minimumAssignedSplitWeight = deltaLakeConfig .getMinimumAssignedSplitWeight ();
125+
126+ DeltaLakeSplitManager splitManager = setupSplitManager (addFileEntries , deltaLakeConfig );
127+ List <DeltaLakeSplit > splits = getSplits (splitManager , deltaLakeConfig );
128+ List <DeltaLakeSplit > expected = ImmutableList .of (
129+ makeSplit (absoluteDecodedParsedFilePath , 0 , 5_000 , fileSize , minimumAssignedSplitWeight ),
130+ makeSplit (absoluteDecodedParsedFilePath , 5_000 , 5_000 , fileSize , minimumAssignedSplitWeight ),
131+ makeSplit (absoluteDecodedParsedFilePath , 10_000 , 5_000 , fileSize , minimumAssignedSplitWeight ),
132+ makeSplit (absoluteDecodedParsedFilePath , 15_000 , 5_000 , fileSize , minimumAssignedSplitWeight ));
133+
134+ assertThat (splits ).isEqualTo (expected );
135+ }
136+
104137 @ Test
105138 public void testSplitSizes ()
106139 throws ExecutionException , InterruptedException
107140 {
108141 long fileSize = 50_000 ;
109- List <AddFileEntry > addFileEntries = ImmutableList .of (addFileEntryOfSize (fileSize ));
142+ List <AddFileEntry > addFileEntries = ImmutableList .of (addFileEntryOfSize (FILE_PATH , fileSize ));
110143 DeltaLakeConfig deltaLakeConfig = new DeltaLakeConfig ()
111144 .setMaxSplitSize (DataSize .ofBytes (20_000 ));
112145 double minimumAssignedSplitWeight = deltaLakeConfig .getMinimumAssignedSplitWeight ();
@@ -115,9 +148,9 @@ public void testSplitSizes()
115148 List <DeltaLakeSplit > splits = getSplits (splitManager , deltaLakeConfig );
116149
117150 List <DeltaLakeSplit > expected = ImmutableList .of (
118- makeSplit (0 , 20_000 , fileSize , minimumAssignedSplitWeight ),
119- makeSplit (20_000 , 20_000 , fileSize , minimumAssignedSplitWeight ),
120- makeSplit (40_000 , 10_000 , fileSize , minimumAssignedSplitWeight ));
151+ makeSplit (FULL_PATH , 0 , 20_000 , fileSize , minimumAssignedSplitWeight ),
152+ makeSplit (FULL_PATH , 20_000 , 20_000 , fileSize , minimumAssignedSplitWeight ),
153+ makeSplit (FULL_PATH , 40_000 , 10_000 , fileSize , minimumAssignedSplitWeight ));
121154
122155 assertThat (splits ).isEqualTo (expected );
123156 }
@@ -128,7 +161,7 @@ public void testSplitsFromMultipleFiles()
128161 {
129162 long firstFileSize = 1_000 ;
130163 long secondFileSize = 20_000 ;
131- List <AddFileEntry > addFileEntries = ImmutableList .of (addFileEntryOfSize (firstFileSize ), addFileEntryOfSize (secondFileSize ));
164+ List <AddFileEntry > addFileEntries = ImmutableList .of (addFileEntryOfSize (FILE_PATH , firstFileSize ), addFileEntryOfSize (FILE_PATH , secondFileSize ));
132165 DeltaLakeConfig deltaLakeConfig = new DeltaLakeConfig ()
133166 .setMaxSplitSize (DataSize .ofBytes (10_000 ));
134167 double minimumAssignedSplitWeight = deltaLakeConfig .getMinimumAssignedSplitWeight ();
@@ -137,9 +170,9 @@ public void testSplitsFromMultipleFiles()
137170
138171 List <DeltaLakeSplit > splits = getSplits (splitManager , deltaLakeConfig );
139172 List <DeltaLakeSplit > expected = ImmutableList .of (
140- makeSplit (0 , 1_000 , firstFileSize , minimumAssignedSplitWeight ),
141- makeSplit (0 , 10_000 , secondFileSize , minimumAssignedSplitWeight ),
142- makeSplit (10_000 , 10_000 , secondFileSize , minimumAssignedSplitWeight ));
173+ makeSplit (FULL_PATH , 0 , 1_000 , firstFileSize , minimumAssignedSplitWeight ),
174+ makeSplit (FULL_PATH , 0 , 10_000 , secondFileSize , minimumAssignedSplitWeight ),
175+ makeSplit (FULL_PATH , 10_000 , 10_000 , secondFileSize , minimumAssignedSplitWeight ));
143176 assertThat (splits ).isEqualTo (expected );
144177 }
145178
@@ -211,15 +244,15 @@ public Stream<AddFileEntry> getActiveFiles(
211244 new DefaultCachingHostAddressProvider ());
212245 }
213246
214- private AddFileEntry addFileEntryOfSize (long fileSize )
247+ private AddFileEntry addFileEntryOfSize (String path , long fileSize )
215248 {
216- return new AddFileEntry (FILE_PATH , ImmutableMap .of (), fileSize , 0 , false , Optional .empty (), Optional .empty (), ImmutableMap .of (), Optional .empty ());
249+ return new AddFileEntry (path , ImmutableMap .of (), fileSize , 0 , false , Optional .empty (), Optional .empty (), ImmutableMap .of (), Optional .empty ());
217250 }
218251
219- private DeltaLakeSplit makeSplit (long start , long splitSize , long fileSize , double minimumAssignedSplitWeight )
252+ private DeltaLakeSplit makeSplit (String path , long start , long splitSize , long fileSize , double minimumAssignedSplitWeight )
220253 {
221254 SplitWeight splitWeight = SplitWeight .fromProportion (clamp ((double ) fileSize / splitSize , minimumAssignedSplitWeight , 1.0 ));
222- return new DeltaLakeSplit (FULL_PATH , start , splitSize , fileSize , Optional .empty (), 0 , Optional .empty (), splitWeight , TupleDomain .all (), ImmutableMap .of ());
255+ return new DeltaLakeSplit (path , start , splitSize , fileSize , Optional .empty (), 0 , Optional .empty (), splitWeight , TupleDomain .all (), ImmutableMap .of ());
223256 }
224257
225258 private List <DeltaLakeSplit > getSplits (DeltaLakeSplitManager splitManager , DeltaLakeConfig deltaLakeConfig )
0 commit comments