Skip to content

Commit

Permalink
Fix the ending order of ParquetJsonColumnConverter
Browse files Browse the repository at this point in the history
  • Loading branch information
nezihyigitbasi authored and cberner committed Mar 6, 2015
1 parent 9575cb1 commit 5e5ca2e
Show file tree
Hide file tree
Showing 3 changed files with 19 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -632,8 +632,8 @@ public void start()
@Override
public void end()
{
jsonConverter.end();
jsonConverter.afterValue();
jsonConverter.end();

nulls[fieldIndex] = false;
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@
import java.sql.Date;
import java.sql.Timestamp;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -217,6 +218,10 @@ public abstract class AbstractTestHiveFileFormats
ImmutableMap.of("test", ImmutableList.<Object>of(new Integer[] {1})),
"{\"test\":[[1]]}"
))
.add(new TestColumn("t_struct_nested", getStandardStructObjectInspector(ImmutableList.of("struct_field") ,
ImmutableList.of(getStandardListObjectInspector(javaStringObjectInspector))), Arrays.asList(Arrays.asList("1", "2", "3")) , "[[\"1\",\"2\",\"3\"]]"))
.add(new TestColumn("t_struct_null", getStandardStructObjectInspector(ImmutableList.of("struct_field") ,
ImmutableList.of(javaStringObjectInspector)), Arrays.asList(new Object[] {null}) , "[null]"))
.build();

protected List<HiveColumnHandle> getColumnHandles(List<TestColumn> testColumns)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,15 +86,25 @@ public void setUp()
public void testRCText()
throws Exception
{
List<TestColumn> testColumns = ImmutableList.copyOf(filter(TEST_COLUMNS, new Predicate<TestColumn>()
{
@Override
public boolean apply(TestColumn testColumn)
{
// TODO: This is a bug in the RC text reader
return !testColumn.getName().equals("t_struct_null");
}
}));

HiveOutputFormat<?, ?> outputFormat = new RCFileOutputFormat();
InputFormat<?, ?> inputFormat = new RCFileInputFormat<>();
@SuppressWarnings("deprecation")
SerDe serde = new ColumnarSerDe();
File file = File.createTempFile("presto_test", "rc-text");
try {
FileSplit split = createTestFile(file.getAbsolutePath(), outputFormat, serde, null, TEST_COLUMNS);
testCursorProvider(new ColumnarTextHiveRecordCursorProvider(), split, inputFormat, serde, TEST_COLUMNS);
testCursorProvider(new GenericHiveRecordCursorProvider(), split, inputFormat, serde, TEST_COLUMNS);
FileSplit split = createTestFile(file.getAbsolutePath(), outputFormat, serde, null, testColumns);
testCursorProvider(new ColumnarTextHiveRecordCursorProvider(), split, inputFormat, serde, testColumns);
testCursorProvider(new GenericHiveRecordCursorProvider(), split, inputFormat, serde, testColumns);
}
finally {
//noinspection ResultOfMethodCallIgnored
Expand Down

0 comments on commit 5e5ca2e

Please sign in to comment.