1717 * under the License.
1818 */
1919
20- package org .apache .iceberg .mr ;
20+ package org .apache .iceberg .mr . mapreduce ;
2121
2222import com .google .common .collect .FluentIterable ;
2323import com .google .common .collect .ImmutableMap ;
24+ import com .google .common .collect .ImmutableSet ;
2425import java .io .File ;
2526import java .io .IOException ;
2627import java .util .ArrayList ;
3435import org .apache .hadoop .mapreduce .TaskAttemptContext ;
3536import org .apache .hadoop .mapreduce .TaskAttemptID ;
3637import org .apache .hadoop .mapreduce .task .TaskAttemptContextImpl ;
38+ import org .apache .iceberg .AssertHelpers ;
3739import org .apache .iceberg .DataFile ;
3840import org .apache .iceberg .DataFiles ;
3941import org .apache .iceberg .FileFormat ;
5153import org .apache .iceberg .data .Record ;
5254import org .apache .iceberg .data .avro .DataWriter ;
5355import org .apache .iceberg .data .parquet .GenericParquetWriter ;
56+ import org .apache .iceberg .expressions .Expressions ;
5457import org .apache .iceberg .hadoop .HadoopCatalog ;
5558import org .apache .iceberg .hadoop .HadoopTables ;
5659import org .apache .iceberg .io .FileAppender ;
5760import org .apache .iceberg .parquet .Parquet ;
61+ import org .apache .iceberg .types .TypeUtil ;
5862import org .apache .iceberg .types .Types ;
5963import org .junit .Assert ;
6064import org .junit .Before ;
6973
7074@ RunWith (Parameterized .class )
7175public class TestIcebergInputFormat {
72- private static final Schema SCHEMA = new Schema (
76+ static final Schema SCHEMA = new Schema (
7377 required (1 , "data" , Types .StringType .get ()),
74- required (3 , "id" , Types .LongType .get ()),
75- required (2 , "date" , Types .StringType .get ()));
78+ required (2 , "id" , Types .LongType .get ()),
79+ required (3 , "date" , Types .StringType .get ()));
7680
77- private static final PartitionSpec SPEC = PartitionSpec .builderFor (SCHEMA )
78- .identity ("date" )
79- .bucket ("id" , 1 )
80- .build ();
81+ static final PartitionSpec SPEC = PartitionSpec .builderFor (SCHEMA )
82+ .identity ("date" )
83+ .bucket ("id" , 1 )
84+ .build ();
8185
8286 @ Rule
8387 public TemporaryFolder temp = new TemporaryFolder ();
@@ -116,7 +120,10 @@ public void testUnpartitionedTable() throws Exception {
116120 table .newAppend ()
117121 .appendFile (dataFile )
118122 .commit ();
119- validate (conf , location .toString (), null , expectedRecords );
123+ Job job = Job .getInstance (conf );
124+ IcebergInputFormat .ConfigBuilder configBuilder = IcebergInputFormat .configure (job );
125+ configBuilder .readFrom (location .toString ());
126+ validate (job , expectedRecords );
120127 }
121128
122129 @ Test
@@ -132,7 +139,136 @@ public void testPartitionedTable() throws Exception {
132139 table .newAppend ()
133140 .appendFile (dataFile )
134141 .commit ();
135- validate (conf , location .toString (), null , expectedRecords );
142+
143+ Job job = Job .getInstance (conf );
144+ IcebergInputFormat .ConfigBuilder configBuilder = IcebergInputFormat .configure (job );
145+ configBuilder .readFrom (location .toString ());
146+ validate (job , expectedRecords );
147+ }
148+
149+ @ Test
150+ public void testFilterExp () throws Exception {
151+ File location = temp .newFolder (format .name ());
152+ Assert .assertTrue (location .delete ());
153+ Table table = tables .create (SCHEMA , SPEC ,
154+ ImmutableMap .of (TableProperties .DEFAULT_FILE_FORMAT , format .name ()),
155+ location .toString ());
156+ List <Record > expectedRecords = RandomGenericData .generate (table .schema (), 2 , 0L );
157+ expectedRecords .get (0 ).set (2 , "2020-03-20" );
158+ expectedRecords .get (1 ).set (2 , "2020-03-20" );
159+ DataFile dataFile1 = writeFile (table , Row .of ("2020-03-20" , 0 ), format , expectedRecords );
160+ DataFile dataFile2 = writeFile (table , Row .of ("2020-03-21" , 0 ), format ,
161+ RandomGenericData .generate (table .schema (), 2 , 0L ));
162+ table .newAppend ()
163+ .appendFile (dataFile1 )
164+ .appendFile (dataFile2 )
165+ .commit ();
166+ Job job = Job .getInstance (conf );
167+ IcebergInputFormat .ConfigBuilder configBuilder = IcebergInputFormat .configure (job );
168+ configBuilder .readFrom (location .toString ())
169+ .filter (Expressions .equal ("date" , "2020-03-20" ));
170+ validate (job , expectedRecords );
171+ }
172+
173+ @ Test
174+ public void testResiduals () throws Exception {
175+ File location = temp .newFolder (format .name ());
176+ Assert .assertTrue (location .delete ());
177+ Table table = tables .create (SCHEMA , SPEC ,
178+ ImmutableMap .of (TableProperties .DEFAULT_FILE_FORMAT , format .name ()),
179+ location .toString ());
180+ List <Record > expectedRecords = RandomGenericData .generate (table .schema (), 2 , 0L );
181+ expectedRecords .get (0 ).set (2 , "2020-03-20" );
182+ expectedRecords .get (1 ).set (2 , "2020-03-20" );
183+ DataFile dataFile = writeFile (table , Row .of ("2020-03-20" , 0 ), format , expectedRecords );
184+ table .newAppend ()
185+ .appendFile (dataFile )
186+ .commit ();
187+ Job job = Job .getInstance (conf );
188+ IcebergInputFormat .ConfigBuilder configBuilder = IcebergInputFormat .configure (job );
189+ configBuilder .readFrom (location .toString ())
190+ .filter (Expressions .and (
191+ Expressions .equal ("date" , "2020-03-20" ),
192+ Expressions .equal ("id" , 0 )));
193+
194+ AssertHelpers .assertThrows (
195+ "Residuals are not evaluated today for Iceberg Generics In memory model" ,
196+ RuntimeException .class , "Filter expression ref(name=\" id\" ) == 0 is not completely satisfied." ,
197+ () -> validate (job , expectedRecords ));
198+ }
199+
200+ @ Test
201+ public void testProjection () throws Exception {
202+ File location = temp .newFolder (format .name ());
203+ Assert .assertTrue (location .delete ());
204+ Schema projectedSchema = TypeUtil .select (SCHEMA , ImmutableSet .of (1 ));
205+ Table table = tables .create (SCHEMA , SPEC ,
206+ ImmutableMap .of (TableProperties .DEFAULT_FILE_FORMAT , format .name ()),
207+ location .toString ());
208+ List <Record > inputRecords = RandomGenericData .generate (table .schema (), 1 , 0L );
209+ DataFile dataFile = writeFile (table , Row .of ("2020-03-20" , 0 ), format , inputRecords );
210+ table .newAppend ()
211+ .appendFile (dataFile )
212+ .commit ();
213+
214+ Job job = Job .getInstance (conf );
215+ IcebergInputFormat .ConfigBuilder configBuilder = IcebergInputFormat .configure (job );
216+ configBuilder
217+ .readFrom (location .toString ())
218+ .project (projectedSchema );
219+ List <Record > outputRecords = readRecords (job .getConfiguration ());
220+ Assert .assertEquals (inputRecords .size (), outputRecords .size ());
221+ Assert .assertEquals (projectedSchema .asStruct (), outputRecords .get (0 ).struct ());
222+ }
223+
224+ @ Test
225+ public void testSnapshotReads () throws Exception {
226+ File location = temp .newFolder (format .name ());
227+ Assert .assertTrue (location .delete ());
228+ Table table = tables .create (SCHEMA , PartitionSpec .unpartitioned (),
229+ ImmutableMap .of (TableProperties .DEFAULT_FILE_FORMAT , format .name ()),
230+ location .toString ());
231+ List <Record > expectedRecords = RandomGenericData .generate (table .schema (), 1 , 0L );
232+ table .newAppend ()
233+ .appendFile (writeFile (table , null , format , expectedRecords ))
234+ .commit ();
235+ long snapshotId = table .currentSnapshot ().snapshotId ();
236+ table .newAppend ()
237+ .appendFile (writeFile (table , null , format , RandomGenericData .generate (table .schema (), 1 , 0L )))
238+ .commit ();
239+
240+ Job job = Job .getInstance (conf );
241+ IcebergInputFormat .ConfigBuilder configBuilder = IcebergInputFormat .configure (job );
242+ configBuilder
243+ .readFrom (location .toString ())
244+ .snapshotId (snapshotId );
245+
246+ validate (job , expectedRecords );
247+ }
248+
249+ @ Test
250+ public void testLocality () throws Exception {
251+ File location = temp .newFolder (format .name ());
252+ Assert .assertTrue (location .delete ());
253+ Table table = tables .create (SCHEMA , PartitionSpec .unpartitioned (),
254+ ImmutableMap .of (TableProperties .DEFAULT_FILE_FORMAT , format .name ()),
255+ location .toString ());
256+ List <Record > expectedRecords = RandomGenericData .generate (table .schema (), 1 , 0L );
257+ table .newAppend ()
258+ .appendFile (writeFile (table , null , format , expectedRecords ))
259+ .commit ();
260+ Job job = Job .getInstance (conf );
261+ IcebergInputFormat .ConfigBuilder configBuilder = IcebergInputFormat .configure (job );
262+ configBuilder .readFrom (location .toString ());
263+
264+ for (InputSplit split : splits (job .getConfiguration ())) {
265+ Assert .assertArrayEquals (IcebergInputFormat .IcebergSplit .ANYWHERE , split .getLocations ());
266+ }
267+
268+ configBuilder .preferLocality ();
269+ for (InputSplit split : splits (job .getConfiguration ())) {
270+ Assert .assertArrayEquals (new String []{"localhost" }, split .getLocations ());
271+ }
136272 }
137273
138274 public static class HadoopCatalogFunc implements Function <Configuration , Catalog > {
@@ -157,22 +293,26 @@ public void testCustomCatalog() throws Exception {
157293 table .newAppend ()
158294 .appendFile (dataFile )
159295 .commit ();
160- validate (conf , tableIdentifier .toString (), HadoopCatalogFunc .class , expectedRecords );
161- }
162296
163- private static void validate (
164- Configuration conf , String path , Class <? extends Function <Configuration , Catalog >> catalogFuncClass ,
165- List <Record > expectedRecords ) throws IOException {
166297 Job job = Job .getInstance (conf );
167298 IcebergInputFormat .ConfigBuilder configBuilder = IcebergInputFormat .configure (job );
168- if (catalogFuncClass != null ) {
169- configBuilder .catalogFunc (catalogFuncClass );
170- }
171- configBuilder .readFrom (path );
299+ configBuilder
300+ .catalogFunc (HadoopCatalogFunc .class )
301+ .readFrom (tableIdentifier .toString ());
302+ validate (job , expectedRecords );
303+ }
304+
305+ private static void validate (Job job , List <Record > expectedRecords ) {
172306 List <Record > actualRecords = readRecords (job .getConfiguration ());
173307 Assert .assertEquals (expectedRecords , actualRecords );
174308 }
175309
310+ private static <T > List <InputSplit > splits (Configuration conf ) {
311+ TaskAttemptContext context = new TaskAttemptContextImpl (conf , new TaskAttemptID ());
312+ IcebergInputFormat <T > icebergInputFormat = new IcebergInputFormat <>();
313+ return icebergInputFormat .getSplits (context );
314+ }
315+
176316 private static <T > List <T > readRecords (Configuration conf ) {
177317 TaskAttemptContext context = new TaskAttemptContextImpl (conf , new TaskAttemptID ());
178318 IcebergInputFormat <T > icebergInputFormat = new IcebergInputFormat <>();
0 commit comments