mirror of https://github.com/apache/druid.git
Support to parse numbers in text-based input formats (#17082)
Text-based input formats like csv and tsv currently parse inputs only as strings, following the RFC4180Parser spec). To workaround this, the web-console and other tools need to further inspect the sample data returned to sample data returned by the Druid sampler API to parse them as numbers. This patch introduces a new optional config, tryParseNumbers, for the csv and tsv input formats. If enabled, any numbers present in the input will be parsed in the following manner -- long data type for integer types and double for floating-point numbers, and if parsing fails for whatever reason, the input is treated as a string. By default, this configuration is set to false, so numeric strings will be treated as strings.
This commit is contained in:
parent
4f137d2700
commit
635e418131
|
@ -132,7 +132,7 @@ public class DelimitedInputFormatBenchmark
|
|||
@Setup(Level.Trial)
|
||||
public void prepareFormat()
|
||||
{
|
||||
format = new DelimitedInputFormat(fromHeader ? null : COLUMNS, null, "\t", null, fromHeader, fromHeader ? 0 : 1);
|
||||
format = new DelimitedInputFormat(fromHeader ? null : COLUMNS, null, "\t", null, fromHeader, fromHeader ? 0 : 1, null);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
|
|
|
@ -125,6 +125,7 @@ Configure the CSV `inputFormat` to load CSV data as follows:
|
|||
| columns | JSON array | Specifies the columns of the data. The columns should be in the same order with the columns of your data. | yes if `findColumnsFromHeader` is false or missing |
|
||||
| findColumnsFromHeader | Boolean | If this is set, the task will find the column names from the header row. Note that `skipHeaderRows` will be applied before finding column names from the header. For example, if you set `skipHeaderRows` to 2 and `findColumnsFromHeader` to true, the task will skip the first two lines and then extract column information from the third line. `columns` will be ignored if this is set to true. | no (default = false if `columns` is set; otherwise null) |
|
||||
| skipHeaderRows | Integer | If this is set, the task will skip the first `skipHeaderRows` rows. | no (default = 0) |
|
||||
| tryParseNumbers| Boolean| If this is set, the task will attempt to parse numeric strings into long or double data type, in that order. This parsing also applies to values separated by `listDelimiter`. If the value cannot be parsed as a number, it is retained as a string. | no (default = false) |
|
||||
|
||||
For example:
|
||||
|
||||
|
@ -150,6 +151,7 @@ Configure the TSV `inputFormat` to load TSV data as follows:
|
|||
| columns | JSON array | Specifies the columns of the data. The columns should be in the same order with the columns of your data. | yes if `findColumnsFromHeader` is false or missing |
|
||||
| findColumnsFromHeader | Boolean | If this is set, the task will find the column names from the header row. Note that `skipHeaderRows` will be applied before finding column names from the header. For example, if you set `skipHeaderRows` to 2 and `findColumnsFromHeader` to true, the task will skip the first two lines and then extract column information from the third line. `columns` will be ignored if this is set to true. | no (default = false if `columns` is set; otherwise null) |
|
||||
| skipHeaderRows | Integer | If this is set, the task will skip the first `skipHeaderRows` rows. | no (default = 0) |
|
||||
| tryParseNumbers| Boolean| If this is set, the task will attempt to parse numeric strings into long or double data type, in that order. This parsing also applies to values separated by `listDelimiter`. If the value cannot be parsed as a number, it is retained as a string. | no (default = false) |
|
||||
|
||||
Be sure to change the `delimiter` to the appropriate delimiter for your data. Like CSV, you must specify the columns and which subset of the columns you want indexed.
|
||||
|
||||
|
|
|
@ -537,7 +537,7 @@ public class OssInputSourceTest extends InitializedNullHandlingTest
|
|||
|
||||
InputSourceReader reader = inputSource.reader(
|
||||
someSchema,
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0),
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null),
|
||||
temporaryFolder.newFolder()
|
||||
);
|
||||
|
||||
|
@ -584,7 +584,7 @@ public class OssInputSourceTest extends InitializedNullHandlingTest
|
|||
|
||||
InputSourceReader reader = inputSource.reader(
|
||||
someSchema,
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0),
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null),
|
||||
temporaryFolder.newFolder()
|
||||
);
|
||||
|
||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.catalog.storage.sql.CatalogManager;
|
||||
import org.apache.druid.catalog.storage.sql.SQLCatalogManager;
|
||||
import org.apache.druid.data.input.InputFormat;
|
||||
import org.apache.druid.data.input.impl.CsvInputFormat;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.metadata.TestDerbyConnector.DerbyConnectorRule;
|
||||
import org.apache.druid.server.security.Access;
|
||||
|
@ -35,8 +33,6 @@ import org.apache.druid.server.security.AuthorizerMapper;
|
|||
import org.apache.druid.server.security.Resource;
|
||||
import org.apache.druid.server.security.ResourceType;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class CatalogTests
|
||||
{
|
||||
public static final String TEST_AUTHORITY = "test";
|
||||
|
@ -74,17 +70,6 @@ public class CatalogTests
|
|||
}
|
||||
}
|
||||
|
||||
public static InputFormat csvFormat()
|
||||
{
|
||||
return new CsvInputFormat(
|
||||
Arrays.asList("x", "y", "z"),
|
||||
null, // listDelimiter
|
||||
false, // hasHeaderRow
|
||||
false, // findColumnsFromHeader
|
||||
0 // skipHeaderRows
|
||||
);
|
||||
}
|
||||
|
||||
public static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper();
|
||||
|
||||
public static class DbFixture
|
||||
|
|
|
@ -406,7 +406,7 @@ public class GoogleCloudStorageInputSourceTest extends InitializedNullHandlingTe
|
|||
|
||||
InputSourceReader reader = inputSource.reader(
|
||||
someSchema,
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0),
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null),
|
||||
null
|
||||
);
|
||||
|
||||
|
@ -453,7 +453,7 @@ public class GoogleCloudStorageInputSourceTest extends InitializedNullHandlingTe
|
|||
|
||||
InputSourceReader reader = inputSource.reader(
|
||||
someSchema,
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0),
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null),
|
||||
null
|
||||
);
|
||||
|
||||
|
|
|
@ -90,7 +90,8 @@ public class HdfsInputSourceTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
false,
|
||||
null,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
|
||||
public static class ConstructorTest
|
||||
|
|
|
@ -712,7 +712,8 @@ public class KafkaInputFormatTest
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
),
|
||||
"kafka.newheader.",
|
||||
"kafka.newkey.key",
|
||||
|
|
|
@ -713,7 +713,8 @@ public class KinesisInputFormatTest
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
),
|
||||
"kinesis.newts.partitionKey",
|
||||
"kinesis.newts.timestamp"
|
||||
|
|
|
@ -327,7 +327,7 @@ public class UriExtractionNamespace implements ExtractionNamespace
|
|||
this.valueColumn,
|
||||
Arrays.toString(columns.toArray())
|
||||
);
|
||||
CSVParser csvParser = new CSVParser(null, columns, hasHeaderRow, skipHeaderRows);
|
||||
CSVParser csvParser = new CSVParser(null, columns, hasHeaderRow, skipHeaderRows, false);
|
||||
csvParser.startFileFromBeginning();
|
||||
this.parser = new DelegateParser(
|
||||
csvParser,
|
||||
|
@ -355,13 +355,13 @@ public class UriExtractionNamespace implements ExtractionNamespace
|
|||
@JsonProperty
|
||||
public String getKeyColumn()
|
||||
{
|
||||
return this.keyColumn;
|
||||
return keyColumn;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getValueColumn()
|
||||
{
|
||||
return this.valueColumn;
|
||||
return valueColumn;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -431,7 +431,8 @@ public class UriExtractionNamespace implements ExtractionNamespace
|
|||
StringUtils.emptyToNullNonDruidDataString(delimiter),
|
||||
StringUtils.emptyToNullNonDruidDataString(listDelimiter),
|
||||
hasHeaderRow,
|
||||
skipHeaderRows
|
||||
skipHeaderRows,
|
||||
false
|
||||
);
|
||||
delegate.startFileFromBeginning();
|
||||
Preconditions.checkArgument(
|
||||
|
|
|
@ -115,6 +115,7 @@ public class UriExtractionNamespaceTest
|
|||
// The third row will parse to data
|
||||
Assert.assertEquals(ImmutableMap.of("val2", "val3"), parser.getParser().parseToMap("val1,val2,val3"));
|
||||
}
|
||||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void testBadCSV()
|
||||
{
|
||||
|
|
|
@ -157,7 +157,7 @@ public class MSQParseExceptionsTest extends MSQTestBase
|
|||
.dataSource(
|
||||
new ExternalDataSource(
|
||||
new LocalInputSource(null, null, ImmutableList.of(toRead), SystemFields.none()),
|
||||
new CsvInputFormat(null, null, null, true, 0),
|
||||
new CsvInputFormat(null, null, null, true, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("timestamp", ColumnType.STRING)
|
||||
.add("agent_category", ColumnType.STRING)
|
||||
|
@ -255,7 +255,7 @@ public class MSQParseExceptionsTest extends MSQTestBase
|
|||
.dataSource(
|
||||
new ExternalDataSource(
|
||||
new LocalInputSource(null, null, ImmutableList.of(toRead), SystemFields.none()),
|
||||
new CsvInputFormat(null, null, null, true, 0),
|
||||
new CsvInputFormat(null, null, null, true, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("timestamp", ColumnType.STRING)
|
||||
.add("agent_category", ColumnType.STRING)
|
||||
|
|
|
@ -1779,7 +1779,7 @@ public class MSQSelectTest extends MSQTestBase
|
|||
.setDataSource(
|
||||
new ExternalDataSource(
|
||||
new InlineInputSource("dim1\nabc\nxyz\ndef\nxyz\nabc\nxyz\nabc\nxyz\ndef\nbbb\naaa"),
|
||||
new CsvInputFormat(null, null, null, true, 0),
|
||||
new CsvInputFormat(null, null, null, true, 0, null),
|
||||
RowSignature.builder().add("dim1", ColumnType.STRING).build()
|
||||
)
|
||||
)
|
||||
|
@ -2376,7 +2376,7 @@ public class MSQSelectTest extends MSQTestBase
|
|||
Collections.nCopies(numFiles, toRead),
|
||||
SystemFields.none()
|
||||
),
|
||||
new CsvInputFormat(null, null, null, true, 0),
|
||||
new CsvInputFormat(null, null, null, true, 0, null),
|
||||
RowSignature.builder().add("timestamp", ColumnType.STRING).build()
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
|
|
|
@ -1015,7 +1015,7 @@ public class S3InputSourceTest extends InitializedNullHandlingTest
|
|||
|
||||
InputSourceReader reader = inputSource.reader(
|
||||
someSchema,
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0),
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null),
|
||||
temporaryFolder.newFolder()
|
||||
);
|
||||
|
||||
|
@ -1063,7 +1063,7 @@ public class S3InputSourceTest extends InitializedNullHandlingTest
|
|||
|
||||
InputSourceReader reader = inputSource.reader(
|
||||
someSchema,
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0),
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null),
|
||||
temporaryFolder.newFolder()
|
||||
);
|
||||
try (CloseableIterator<InputRow> readerIterator = reader.read()) {
|
||||
|
@ -1111,7 +1111,7 @@ public class S3InputSourceTest extends InitializedNullHandlingTest
|
|||
|
||||
InputSourceReader reader = inputSource.reader(
|
||||
someSchema,
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0),
|
||||
new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null),
|
||||
temporaryFolder.newFolder()
|
||||
);
|
||||
|
||||
|
|
|
@ -970,7 +970,8 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
|
|||
"|",
|
||||
null,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
),
|
||||
appendToExisting,
|
||||
null
|
||||
|
|
|
@ -163,7 +163,8 @@ public class IndexTaskTest extends IngestionTestBase
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
|
||||
private static final DataSchema DATA_SCHEMA =
|
||||
|
@ -473,7 +474,7 @@ public class IndexTaskTest extends IngestionTestBase
|
|||
indexIngestionSpec = createIngestionSpec(
|
||||
DEFAULT_TIMESTAMP_SPEC,
|
||||
dimensionsSpec,
|
||||
new CsvInputFormat(columns, listDelimiter, null, false, 0),
|
||||
new CsvInputFormat(columns, listDelimiter, null, false, 0, null),
|
||||
transformSpec,
|
||||
null,
|
||||
tuningConfig,
|
||||
|
@ -901,7 +902,7 @@ public class IndexTaskTest extends IngestionTestBase
|
|||
ingestionSpec = createIngestionSpec(
|
||||
timestampSpec,
|
||||
DimensionsSpec.EMPTY,
|
||||
new CsvInputFormat(null, null, null, true, 0),
|
||||
new CsvInputFormat(null, null, null, true, 0, null),
|
||||
null,
|
||||
null,
|
||||
tuningConfig,
|
||||
|
@ -941,7 +942,7 @@ public class IndexTaskTest extends IngestionTestBase
|
|||
ingestionSpec = createIngestionSpec(
|
||||
timestampSpec,
|
||||
DimensionsSpec.EMPTY,
|
||||
new CsvInputFormat(columns, null, null, true, 0),
|
||||
new CsvInputFormat(columns, null, null, true, 0, null),
|
||||
null,
|
||||
null,
|
||||
tuningConfig,
|
||||
|
@ -1341,7 +1342,7 @@ public class IndexTaskTest extends IngestionTestBase
|
|||
parseExceptionIgnoreSpec = createIngestionSpec(
|
||||
timestampSpec,
|
||||
DimensionsSpec.EMPTY,
|
||||
new CsvInputFormat(columns, null, null, true, 0),
|
||||
new CsvInputFormat(columns, null, null, true, 0, null),
|
||||
null,
|
||||
null,
|
||||
tuningConfig,
|
||||
|
@ -1391,7 +1392,7 @@ public class IndexTaskTest extends IngestionTestBase
|
|||
indexIngestionSpec = createIngestionSpec(
|
||||
timestampSpec,
|
||||
DimensionsSpec.EMPTY,
|
||||
new CsvInputFormat(columns, null, null, true, 0),
|
||||
new CsvInputFormat(columns, null, null, true, 0, null),
|
||||
null,
|
||||
null,
|
||||
tuningConfig,
|
||||
|
@ -1632,7 +1633,7 @@ public class IndexTaskTest extends IngestionTestBase
|
|||
ingestionSpec = createIngestionSpec(
|
||||
timestampSpec,
|
||||
dimensionsSpec,
|
||||
new CsvInputFormat(columns, null, null, true, 0),
|
||||
new CsvInputFormat(columns, null, null, true, 0, null),
|
||||
null,
|
||||
null,
|
||||
tuningConfig,
|
||||
|
@ -1751,7 +1752,7 @@ public class IndexTaskTest extends IngestionTestBase
|
|||
ingestionSpec = createIngestionSpec(
|
||||
timestampSpec,
|
||||
dimensionsSpec,
|
||||
new CsvInputFormat(columns, null, null, true, 0),
|
||||
new CsvInputFormat(columns, null, null, true, 0, null),
|
||||
null,
|
||||
null,
|
||||
tuningConfig,
|
||||
|
@ -1845,7 +1846,7 @@ public class IndexTaskTest extends IngestionTestBase
|
|||
ingestionSpec = createIngestionSpec(
|
||||
DEFAULT_TIMESTAMP_SPEC,
|
||||
DimensionsSpec.EMPTY,
|
||||
new CsvInputFormat(null, null, null, true, 0),
|
||||
new CsvInputFormat(null, null, null, true, 0, null),
|
||||
null,
|
||||
null,
|
||||
tuningConfig,
|
||||
|
@ -1915,7 +1916,7 @@ public class IndexTaskTest extends IngestionTestBase
|
|||
ingestionSpec = createIngestionSpec(
|
||||
DEFAULT_TIMESTAMP_SPEC,
|
||||
DimensionsSpec.EMPTY,
|
||||
new CsvInputFormat(columns, null, null, true, 0),
|
||||
new CsvInputFormat(columns, null, null, true, 0, null),
|
||||
null,
|
||||
null,
|
||||
tuningConfig,
|
||||
|
|
|
@ -313,7 +313,8 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest
|
|||
csvParseSpec.getListDelimiter(),
|
||||
getColumnsFromHeader ? null : true,
|
||||
getColumnsFromHeader ? true : null,
|
||||
csvParseSpec.getSkipHeaderRows()
|
||||
csvParseSpec.getSkipHeaderRows(),
|
||||
null
|
||||
);
|
||||
} else if (parseSpec instanceof DelimitedParseSpec) {
|
||||
DelimitedParseSpec delimitedParseSpec = (DelimitedParseSpec) parseSpec;
|
||||
|
@ -324,7 +325,8 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest
|
|||
delimitedParseSpec.getDelimiter(),
|
||||
getColumnsFromHeader ? null : true,
|
||||
getColumnsFromHeader ? true : null,
|
||||
delimitedParseSpec.getSkipHeaderRows()
|
||||
delimitedParseSpec.getSkipHeaderRows(),
|
||||
null
|
||||
);
|
||||
} else if (parseSpec instanceof RegexParseSpec) {
|
||||
RegexParseSpec regexParseSpec = (RegexParseSpec) parseSpec;
|
||||
|
|
|
@ -163,7 +163,8 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
public static final ParallelIndexTuningConfig DEFAULT_TUNING_CONFIG_FOR_PARALLEL_INDEXING =
|
||||
TuningConfigBuilder.forParallelIndexTask()
|
||||
|
|
|
@ -61,7 +61,8 @@ public class HashPartitionAdjustingCorePartitionSizeTest extends AbstractMultiPh
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
private static final Interval INTERVAL_TO_INDEX = Intervals.of("2020-01-01/P1M");
|
||||
|
||||
|
|
|
@ -84,7 +84,8 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
private static final Interval INTERVAL_TO_INDEX = Intervals.of("2017-12/P1M");
|
||||
private static final String INPUT_FILTER = "test_*";
|
||||
|
|
|
@ -79,7 +79,8 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
private static final Interval INTERVAL_TO_INDEX = Intervals.of("2017-12/P1M");
|
||||
|
||||
|
|
|
@ -182,7 +182,7 @@ public class ParallelIndexSupervisorTaskSerdeTest
|
|||
|
||||
private final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
|
||||
new LocalInputSource(new File("tmp"), "test_*"),
|
||||
new CsvInputFormat(Arrays.asList("ts", "dim", "val"), null, null, false, 0),
|
||||
new CsvInputFormat(Arrays.asList("ts", "dim", "val"), null, null, false, 0, null),
|
||||
false,
|
||||
null
|
||||
);
|
||||
|
|
|
@ -64,7 +64,8 @@ public class PartialCompactionTest extends AbstractMultiPhaseParallelIndexingTes
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
private static final Interval INTERVAL_TO_INDEX = Intervals.of("2017-12/P1M");
|
||||
|
||||
|
|
|
@ -61,7 +61,8 @@ public class RangePartitionAdjustingCorePartitionSizeTest extends AbstractMultiP
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
private static final Interval INTERVAL_TO_INDEX = Intervals.of("2020-01-01/P1M");
|
||||
|
||||
|
|
|
@ -110,7 +110,8 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
|
|||
LIST_DELIMITER,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
|
||||
@Parameterized.Parameters(name = "{0}, useInputFormatApi={1}, maxNumConcurrentSubTasks={2}, useMultiValueDim={3}, intervalToIndex={4}")
|
||||
|
|
|
@ -84,7 +84,8 @@ public class RangePartitionTaskKillTest extends AbstractMultiPhaseParallelIndexi
|
|||
LIST_DELIMITER,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
|
||||
private File inputDir;
|
||||
|
|
|
@ -61,7 +61,7 @@ public class CsvInputSourceSamplerTest extends InitializedNullHandlingTest
|
|||
"Michael,Jackson,,Male"
|
||||
);
|
||||
final InputSource inputSource = new InlineInputSource(String.join("\n", strCsvRows));
|
||||
final InputFormat inputFormat = new CsvInputFormat(null, null, null, true, 0);
|
||||
final InputFormat inputFormat = new CsvInputFormat(null, null, null, true, 0, null);
|
||||
final InputSourceSampler inputSourceSampler = new InputSourceSampler(new DefaultObjectMapper());
|
||||
|
||||
final SamplerResponse response = inputSourceSampler.sample(
|
||||
|
|
|
@ -1460,7 +1460,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
|
|||
case STR_JSON:
|
||||
return new JsonInputFormat(null, null, null, null, null);
|
||||
case STR_CSV:
|
||||
return new CsvInputFormat(ImmutableList.of("t", "dim1", "dim2", "met1"), null, null, false, 0);
|
||||
return new CsvInputFormat(ImmutableList.of("t", "dim1", "dim2", "met1"), null, null, false, 0, null);
|
||||
default:
|
||||
throw new IAE("Unknown parser type: %s", parserType);
|
||||
}
|
||||
|
|
|
@ -80,7 +80,7 @@ public class RecordSupplierInputSourceTest extends InitializedNullHandlingTest
|
|||
final List<String> colNames = IntStream.range(0, NUM_COLS)
|
||||
.mapToObj(i -> StringUtils.format("col_%d", i))
|
||||
.collect(Collectors.toList());
|
||||
final InputFormat inputFormat = new CsvInputFormat(colNames, null, null, false, 0);
|
||||
final InputFormat inputFormat = new CsvInputFormat(colNames, null, null, false, 0, null);
|
||||
final InputSourceReader reader = inputSource.reader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("col_0", "auto", null),
|
||||
|
@ -114,7 +114,7 @@ public class RecordSupplierInputSourceTest extends InitializedNullHandlingTest
|
|||
final List<String> colNames = IntStream.range(0, NUM_COLS)
|
||||
.mapToObj(i -> StringUtils.format("col_%d", i))
|
||||
.collect(Collectors.toList());
|
||||
final InputFormat inputFormat = new CsvInputFormat(colNames, null, null, false, 0);
|
||||
final InputFormat inputFormat = new CsvInputFormat(colNames, null, null, false, 0, null);
|
||||
final InputSourceReader reader = inputSource.reader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("col_0", "auto", null),
|
||||
|
|
|
@ -380,7 +380,7 @@ public class SeekableStreamIndexTaskRunnerAuthTest
|
|||
false,
|
||||
DateTimes.nowUtc().minusDays(2),
|
||||
DateTimes.nowUtc(),
|
||||
new CsvInputFormat(null, null, true, null, 0)
|
||||
new CsvInputFormat(null, null, true, null, 0, null)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -93,7 +93,7 @@ public class CSVParseSpec extends ParseSpec
|
|||
@Override
|
||||
public Parser<String, Object> makeParser()
|
||||
{
|
||||
return new CSVParser(listDelimiter, columns, hasHeaderRow, skipHeaderRows);
|
||||
return new CSVParser(listDelimiter, columns, hasHeaderRow, skipHeaderRows, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -47,10 +47,11 @@ public class CsvInputFormat extends FlatTextInputFormat
|
|||
@JsonProperty("listDelimiter") @Nullable String listDelimiter,
|
||||
@Deprecated @JsonProperty("hasHeaderRow") @Nullable Boolean hasHeaderRow,
|
||||
@JsonProperty("findColumnsFromHeader") @Nullable Boolean findColumnsFromHeader,
|
||||
@JsonProperty("skipHeaderRows") int skipHeaderRows
|
||||
@JsonProperty("skipHeaderRows") int skipHeaderRows,
|
||||
@JsonProperty("tryParseNumbers") @Nullable Boolean tryParseNumbers
|
||||
)
|
||||
{
|
||||
super(columns, listDelimiter, String.valueOf(SEPARATOR), hasHeaderRow, findColumnsFromHeader, skipHeaderRows);
|
||||
super(columns, listDelimiter, String.valueOf(SEPARATOR), hasHeaderRow, findColumnsFromHeader, skipHeaderRows, tryParseNumbers);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -80,7 +81,8 @@ public class CsvInputFormat extends FlatTextInputFormat
|
|||
isFindColumnsFromHeader(),
|
||||
getSkipHeaderRows(),
|
||||
line -> Arrays.asList(parser.parseLine(StringUtils.fromUtf8(line))),
|
||||
useListBasedInputRows()
|
||||
useListBasedInputRows(),
|
||||
shouldTryParseNumbers()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -52,7 +52,8 @@ public class DelimitedInputFormat extends FlatTextInputFormat
|
|||
@JsonProperty("delimiter") @Nullable String delimiter,
|
||||
@Deprecated @JsonProperty("hasHeaderRow") @Nullable Boolean hasHeaderRow,
|
||||
@JsonProperty("findColumnsFromHeader") @Nullable Boolean findColumnsFromHeader,
|
||||
@JsonProperty("skipHeaderRows") int skipHeaderRows
|
||||
@JsonProperty("skipHeaderRows") int skipHeaderRows,
|
||||
@JsonProperty("tryParseNumbers") @Nullable Boolean tryParseNumbers
|
||||
)
|
||||
{
|
||||
super(
|
||||
|
@ -61,7 +62,8 @@ public class DelimitedInputFormat extends FlatTextInputFormat
|
|||
delimiter == null ? DEFAULT_DELIMITER : delimiter,
|
||||
hasHeaderRow,
|
||||
findColumnsFromHeader,
|
||||
skipHeaderRows
|
||||
skipHeaderRows,
|
||||
tryParseNumbers
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -85,7 +87,8 @@ public class DelimitedInputFormat extends FlatTextInputFormat
|
|||
getDelimiter(),
|
||||
useListBasedInputRows() ? getColumns().size() : DelimitedBytes.UNKNOWN_FIELD_COUNT
|
||||
),
|
||||
useListBasedInputRows()
|
||||
useListBasedInputRows(),
|
||||
shouldTryParseNumbers()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -69,18 +69,6 @@ public class DelimitedParseSpec extends ParseSpec
|
|||
}
|
||||
}
|
||||
|
||||
@Deprecated
|
||||
public DelimitedParseSpec(
|
||||
TimestampSpec timestampSpec,
|
||||
DimensionsSpec dimensionsSpec,
|
||||
String delimiter,
|
||||
String listDelimiter,
|
||||
List<String> columns
|
||||
)
|
||||
{
|
||||
this(timestampSpec, dimensionsSpec, delimiter, listDelimiter, columns, false, 0);
|
||||
}
|
||||
|
||||
@JsonProperty("delimiter")
|
||||
public String getDelimiter()
|
||||
{
|
||||
|
@ -119,7 +107,8 @@ public class DelimitedParseSpec extends ParseSpec
|
|||
listDelimiter,
|
||||
columns,
|
||||
hasHeaderRow,
|
||||
skipHeaderRows
|
||||
skipHeaderRows,
|
||||
false
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -51,7 +51,7 @@ public class DelimitedValueReader extends TextReader.Bytes
|
|||
{
|
||||
private final boolean findColumnsFromHeader;
|
||||
private final int skipHeaderRows;
|
||||
private final Function<String, Object> multiValueFunction;
|
||||
private final Function<String, Object> transformationFunction;
|
||||
private final DelimitedValueParser parser;
|
||||
|
||||
/**
|
||||
|
@ -82,14 +82,19 @@ public class DelimitedValueReader extends TextReader.Bytes
|
|||
boolean findColumnsFromHeader,
|
||||
int skipHeaderRows,
|
||||
DelimitedValueParser parser,
|
||||
boolean useListBasedInputRows
|
||||
boolean useListBasedInputRows,
|
||||
boolean tryParseNumbers
|
||||
)
|
||||
{
|
||||
super(inputRowSchema, source);
|
||||
this.findColumnsFromHeader = findColumnsFromHeader;
|
||||
this.skipHeaderRows = skipHeaderRows;
|
||||
final String finalListDelimeter = listDelimiter == null ? Parsers.DEFAULT_LIST_DELIMITER : listDelimiter;
|
||||
this.multiValueFunction = ParserUtils.getMultiValueFunction(finalListDelimeter, Splitter.on(finalListDelimeter));
|
||||
this.transformationFunction = ParserUtils.getTransformationFunction(
|
||||
finalListDelimeter,
|
||||
Splitter.on(finalListDelimeter),
|
||||
tryParseNumbers
|
||||
);
|
||||
|
||||
if (!findColumnsFromHeader && columns != null) {
|
||||
// If findColumnsFromHeader, inputRowSignature will be set later.
|
||||
|
@ -134,7 +139,7 @@ public class DelimitedValueReader extends TextReader.Bytes
|
|||
private List<Object> readLineAsList(byte[] line) throws IOException
|
||||
{
|
||||
final List<String> parsed = parser.parseLine(line);
|
||||
return new ArrayList<>(Lists.transform(parsed, multiValueFunction));
|
||||
return new ArrayList<>(Lists.transform(parsed, transformationFunction));
|
||||
}
|
||||
|
||||
private Map<String, Object> readLineAsMap(byte[] line) throws IOException
|
||||
|
@ -142,7 +147,7 @@ public class DelimitedValueReader extends TextReader.Bytes
|
|||
final List<String> parsed = parser.parseLine(line);
|
||||
return Utils.zipMapPartial(
|
||||
Preconditions.checkNotNull(inputRowSignature, "inputRowSignature").getColumnNames(),
|
||||
Iterables.transform(parsed, multiValueFunction)
|
||||
Iterables.transform(parsed, transformationFunction)
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -46,6 +46,7 @@ public abstract class FlatTextInputFormat implements InputFormat
|
|||
private final String delimiter;
|
||||
private final boolean findColumnsFromHeader;
|
||||
private final int skipHeaderRows;
|
||||
private final boolean tryParseNumbers;
|
||||
|
||||
FlatTextInputFormat(
|
||||
@Nullable List<String> columns,
|
||||
|
@ -53,7 +54,8 @@ public abstract class FlatTextInputFormat implements InputFormat
|
|||
String delimiter,
|
||||
@Nullable Boolean hasHeaderRow,
|
||||
@Nullable Boolean findColumnsFromHeader,
|
||||
int skipHeaderRows
|
||||
int skipHeaderRows,
|
||||
@Nullable Boolean tryParseNumbers
|
||||
)
|
||||
{
|
||||
this.columns = columns == null ? Collections.emptyList() : columns;
|
||||
|
@ -79,6 +81,8 @@ public abstract class FlatTextInputFormat implements InputFormat
|
|||
"Cannot have same delimiter and list delimiter of [%s]",
|
||||
delimiter
|
||||
);
|
||||
this.tryParseNumbers = tryParseNumbers == null ? false : tryParseNumbers;
|
||||
|
||||
if (!this.columns.isEmpty()) {
|
||||
for (String column : this.columns) {
|
||||
Preconditions.checkArgument(
|
||||
|
@ -131,6 +135,13 @@ public abstract class FlatTextInputFormat implements InputFormat
|
|||
return skipHeaderRows;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
|
||||
public boolean shouldTryParseNumbers()
|
||||
{
|
||||
return tryParseNumbers;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getWeightedSize(String path, long size)
|
||||
{
|
||||
|
|
|
@ -43,7 +43,7 @@ public class RegexReader extends TextReader.Strings
|
|||
{
|
||||
private final String pattern;
|
||||
private final Pattern compiledPattern;
|
||||
private final Function<String, Object> multiValueFunction;
|
||||
private final Function<String, Object> transformationFunction;
|
||||
|
||||
private List<String> columns;
|
||||
|
||||
|
@ -60,7 +60,11 @@ public class RegexReader extends TextReader.Strings
|
|||
this.pattern = pattern;
|
||||
this.compiledPattern = compiledPattern;
|
||||
final String finalListDelimeter = listDelimiter == null ? Parsers.DEFAULT_LIST_DELIMITER : listDelimiter;
|
||||
this.multiValueFunction = ParserUtils.getMultiValueFunction(finalListDelimeter, Splitter.on(finalListDelimeter));
|
||||
this.transformationFunction = ParserUtils.getTransformationFunction(
|
||||
finalListDelimeter,
|
||||
Splitter.on(finalListDelimeter),
|
||||
false
|
||||
);
|
||||
this.columns = columns;
|
||||
}
|
||||
|
||||
|
@ -94,7 +98,7 @@ public class RegexReader extends TextReader.Strings
|
|||
columns = ParserUtils.generateFieldNames(matcher.groupCount());
|
||||
}
|
||||
|
||||
return Utils.zipMapPartial(columns, Iterables.transform(values, multiValueFunction));
|
||||
return Utils.zipMapPartial(columns, Iterables.transform(values, transformationFunction));
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new ParseException(line, e, "Unable to parse row [%s]", line);
|
||||
|
|
|
@ -52,7 +52,7 @@ public abstract class AbstractFlatTextFormatParser implements Parser<String, Obj
|
|||
}
|
||||
|
||||
private final String listDelimiter;
|
||||
private final Function<String, Object> valueFunction;
|
||||
private final Function<String, Object> transformationFunction;
|
||||
private final boolean hasHeaderRow;
|
||||
private final int maxSkipHeaderRows;
|
||||
|
||||
|
@ -64,11 +64,16 @@ public abstract class AbstractFlatTextFormatParser implements Parser<String, Obj
|
|||
public AbstractFlatTextFormatParser(
|
||||
@Nullable final String listDelimiter,
|
||||
final boolean hasHeaderRow,
|
||||
final int maxSkipHeaderRows
|
||||
final int maxSkipHeaderRows,
|
||||
final boolean tryParseNumbers
|
||||
)
|
||||
{
|
||||
this.listDelimiter = listDelimiter != null ? listDelimiter : Parsers.DEFAULT_LIST_DELIMITER;
|
||||
this.valueFunction = ParserUtils.getMultiValueFunction(this.listDelimiter, Splitter.on(this.listDelimiter));
|
||||
this.transformationFunction = ParserUtils.getTransformationFunction(
|
||||
this.listDelimiter,
|
||||
Splitter.on(this.listDelimiter),
|
||||
tryParseNumbers
|
||||
);
|
||||
|
||||
this.hasHeaderRow = hasHeaderRow;
|
||||
this.maxSkipHeaderRows = maxSkipHeaderRows;
|
||||
|
@ -142,7 +147,7 @@ public abstract class AbstractFlatTextFormatParser implements Parser<String, Obj
|
|||
setFieldNames(ParserUtils.generateFieldNames(values.size()));
|
||||
}
|
||||
|
||||
return Utils.zipMapPartial(fieldNames, Iterables.transform(values, valueFunction));
|
||||
return Utils.zipMapPartial(fieldNames, Iterables.transform(values, transformationFunction));
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new ParseException(input, e, "Unable to parse row [%s]", input);
|
||||
|
|
|
@ -35,20 +35,22 @@ public class CSVParser extends AbstractFlatTextFormatParser
|
|||
public CSVParser(
|
||||
@Nullable final String listDelimiter,
|
||||
final boolean hasHeaderRow,
|
||||
final int maxSkipHeaderRows
|
||||
final int maxSkipHeaderRows,
|
||||
final boolean tryParseNumbers
|
||||
)
|
||||
{
|
||||
super(listDelimiter, hasHeaderRow, maxSkipHeaderRows);
|
||||
super(listDelimiter, hasHeaderRow, maxSkipHeaderRows, tryParseNumbers);
|
||||
}
|
||||
|
||||
public CSVParser(
|
||||
@Nullable final String listDelimiter,
|
||||
final Iterable<String> fieldNames,
|
||||
final boolean hasHeaderRow,
|
||||
final int maxSkipHeaderRows
|
||||
final int maxSkipHeaderRows,
|
||||
final boolean tryParseNumbers
|
||||
)
|
||||
{
|
||||
this(listDelimiter, hasHeaderRow, maxSkipHeaderRows);
|
||||
this(listDelimiter, hasHeaderRow, maxSkipHeaderRows, tryParseNumbers);
|
||||
|
||||
setFieldNames(fieldNames);
|
||||
}
|
||||
|
@ -56,7 +58,7 @@ public class CSVParser extends AbstractFlatTextFormatParser
|
|||
@VisibleForTesting
|
||||
CSVParser(@Nullable final String listDelimiter, final String header)
|
||||
{
|
||||
this(listDelimiter, false, 0);
|
||||
this(listDelimiter, false, 0, false);
|
||||
|
||||
setFieldNames(header);
|
||||
}
|
||||
|
|
|
@ -35,10 +35,11 @@ public class DelimitedParser extends AbstractFlatTextFormatParser
|
|||
@Nullable final String delimiter,
|
||||
@Nullable final String listDelimiter,
|
||||
final boolean hasHeaderRow,
|
||||
final int maxSkipHeaderRows
|
||||
final int maxSkipHeaderRows,
|
||||
final boolean tryParseNumbers
|
||||
)
|
||||
{
|
||||
super(listDelimiter, hasHeaderRow, maxSkipHeaderRows);
|
||||
super(listDelimiter, hasHeaderRow, maxSkipHeaderRows, tryParseNumbers);
|
||||
final String finalDelimiter = delimiter != null ? delimiter : FlatTextFormat.DELIMITED.getDefaultDelimiter();
|
||||
|
||||
Preconditions.checkState(
|
||||
|
@ -55,10 +56,11 @@ public class DelimitedParser extends AbstractFlatTextFormatParser
|
|||
@Nullable final String listDelimiter,
|
||||
final Iterable<String> fieldNames,
|
||||
final boolean hasHeaderRow,
|
||||
final int maxSkipHeaderRows
|
||||
final int maxSkipHeaderRows,
|
||||
final boolean tryParseNumbers
|
||||
)
|
||||
{
|
||||
this(delimiter, listDelimiter, hasHeaderRow, maxSkipHeaderRows);
|
||||
this(delimiter, listDelimiter, hasHeaderRow, maxSkipHeaderRows, tryParseNumbers);
|
||||
|
||||
setFieldNames(fieldNames);
|
||||
}
|
||||
|
@ -66,7 +68,7 @@ public class DelimitedParser extends AbstractFlatTextFormatParser
|
|||
@VisibleForTesting
|
||||
DelimitedParser(@Nullable final String delimiter, @Nullable final String listDelimiter, final String header)
|
||||
{
|
||||
this(delimiter, listDelimiter, false, 0);
|
||||
this(delimiter, listDelimiter, false, 0, false);
|
||||
|
||||
setFieldNames(header);
|
||||
}
|
||||
|
|
|
@ -22,6 +22,9 @@ package org.apache.druid.java.util.common.parsers;
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.primitives.Doubles;
|
||||
import com.google.common.primitives.Longs;
|
||||
import org.apache.commons.lang.math.NumberUtils;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
||||
|
@ -52,22 +55,62 @@ public class ParserUtils
|
|||
}
|
||||
}
|
||||
|
||||
public static Function<String, Object> getMultiValueFunction(
|
||||
/**
|
||||
* @return a transformation function on an input value. The function performs the following transformations on the input string:
|
||||
* <li> Splits it into multiple values using the {@code listSplitter} if the {@code list delimiter} is present in the input. </li>
|
||||
* <li> If {@code tryParseNumbers} is true, the function will also attempt to parse any numeric values present in the input:
|
||||
* integers as {@code Long} and floating-point numbers as {@code Double}. If the input is not a number or parsing fails, the input
|
||||
* is returned as-is as a string. </li>
|
||||
*/
|
||||
public static Function<String, Object> getTransformationFunction(
|
||||
final String listDelimiter,
|
||||
final Splitter listSplitter
|
||||
final Splitter listSplitter,
|
||||
final boolean tryParseNumbers
|
||||
)
|
||||
{
|
||||
return (input) -> {
|
||||
if (input != null && input.contains(listDelimiter)) {
|
||||
return StreamSupport.stream(listSplitter.split(input).spliterator(), false)
|
||||
.map(NullHandling::emptyToNullIfNeeded)
|
||||
.collect(Collectors.toList());
|
||||
} else {
|
||||
if (input == null) {
|
||||
return NullHandling.emptyToNullIfNeeded(input);
|
||||
}
|
||||
|
||||
if (input.contains(listDelimiter)) {
|
||||
return StreamSupport.stream(listSplitter.split(input).spliterator(), false)
|
||||
.map(NullHandling::emptyToNullIfNeeded)
|
||||
.map(value -> tryParseNumbers ? ParserUtils.tryParseStringAsNumber(value) : value)
|
||||
.collect(Collectors.toList());
|
||||
} else {
|
||||
return tryParseNumbers ?
|
||||
tryParseStringAsNumber(input) :
|
||||
NullHandling.emptyToNullIfNeeded(input);
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempts to parse the input string into a numeric value, if applicable. If the input is a number, the method first
|
||||
* tries to parse the input number as a {@code Long}. If parsing as a {@code Long} fails, it then attempts to parse
|
||||
* the input number as a {@code Double}. For all other scenarios, the input is returned as-is as a {@code String} type.
|
||||
*/
|
||||
@Nullable
|
||||
private static Object tryParseStringAsNumber(@Nullable final String input)
|
||||
{
|
||||
if (!NumberUtils.isNumber(input)) {
|
||||
return NullHandling.emptyToNullIfNeeded(input);
|
||||
}
|
||||
|
||||
final Long l = Longs.tryParse(input);
|
||||
if (l != null) {
|
||||
return l;
|
||||
}
|
||||
final Double d = Doubles.tryParse(input);
|
||||
if (d != null) {
|
||||
return d;
|
||||
}
|
||||
// fall back to given input if we cannot parse the input as a Long & Double for whatever reason
|
||||
return input;
|
||||
}
|
||||
|
||||
public static ArrayList<String> generateFieldNames(int length)
|
||||
{
|
||||
final ArrayList<String> names = new ArrayList<>(length);
|
||||
|
|
|
@ -44,7 +44,7 @@ public class CsvInputFormatTest extends InitializedNullHandlingTest
|
|||
public void testSerde() throws IOException
|
||||
{
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), "|", null, true, 10);
|
||||
final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), "|", null, true, 10, null);
|
||||
final byte[] bytes = mapper.writeValueAsBytes(format);
|
||||
final CsvInputFormat fromJson = (CsvInputFormat) mapper.readValue(bytes, InputFormat.class);
|
||||
Assert.assertEquals(format, fromJson);
|
||||
|
@ -127,12 +127,23 @@ public class CsvInputFormatTest extends InitializedNullHandlingTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeserializeWithTryParseNumbers() throws IOException
|
||||
{
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
final CsvInputFormat inputFormat = (CsvInputFormat) mapper.readValue(
|
||||
"{\"type\":\"csv\",\"hasHeaderRow\":true,\"tryParseNumbers\":true}",
|
||||
InputFormat.class
|
||||
);
|
||||
Assert.assertTrue(inputFormat.shouldTryParseNumbers());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testComma()
|
||||
{
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Column[a,] cannot have the delimiter[,] in its name");
|
||||
new CsvInputFormat(Collections.singletonList("a,"), "|", null, false, 0);
|
||||
new CsvInputFormat(Collections.singletonList("a,"), "|", null, false, 0, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -140,20 +151,20 @@ public class CsvInputFormatTest extends InitializedNullHandlingTest
|
|||
{
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Cannot have same delimiter and list delimiter of [,]");
|
||||
new CsvInputFormat(Collections.singletonList("a\t"), ",", null, false, 0);
|
||||
new CsvInputFormat(Collections.singletonList("a\t"), ",", null, false, 0, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFindColumnsFromHeaderWithColumnsReturningItsValue()
|
||||
{
|
||||
final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), null, null, true, 0);
|
||||
final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), null, null, true, 0, null);
|
||||
Assert.assertTrue(format.isFindColumnsFromHeader());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFindColumnsFromHeaderWithMissingColumnsReturningItsValue()
|
||||
{
|
||||
final CsvInputFormat format = new CsvInputFormat(null, null, null, true, 0);
|
||||
final CsvInputFormat format = new CsvInputFormat(null, null, null, true, 0, null);
|
||||
Assert.assertTrue(format.isFindColumnsFromHeader());
|
||||
}
|
||||
|
||||
|
@ -162,13 +173,13 @@ public class CsvInputFormatTest extends InitializedNullHandlingTest
|
|||
{
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Either [columns] or [findColumnsFromHeader] must be set");
|
||||
new CsvInputFormat(null, null, null, null, 0);
|
||||
new CsvInputFormat(null, null, null, null, 0, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMissingFindColumnsFromHeaderWithColumnsReturningFalse()
|
||||
{
|
||||
final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), null, null, null, 0);
|
||||
final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), null, null, null, 0, null);
|
||||
Assert.assertFalse(format.isFindColumnsFromHeader());
|
||||
}
|
||||
|
||||
|
@ -177,20 +188,20 @@ public class CsvInputFormatTest extends InitializedNullHandlingTest
|
|||
{
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Cannot accept both [findColumnsFromHeader] and [hasHeaderRow]");
|
||||
new CsvInputFormat(null, null, true, false, 0);
|
||||
new CsvInputFormat(null, null, true, false, 0, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHasHeaderRowWithMissingColumnsReturningItsValue()
|
||||
{
|
||||
final CsvInputFormat format = new CsvInputFormat(null, null, true, null, 0);
|
||||
final CsvInputFormat format = new CsvInputFormat(null, null, true, null, 0, null);
|
||||
Assert.assertTrue(format.isFindColumnsFromHeader());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getWeightedSize_withoutCompression()
|
||||
{
|
||||
final CsvInputFormat format = new CsvInputFormat(null, null, true, null, 0);
|
||||
final CsvInputFormat format = new CsvInputFormat(null, null, true, null, 0, null);
|
||||
final long unweightedSize = 100L;
|
||||
Assert.assertEquals(unweightedSize, format.getWeightedSize("file.csv", unweightedSize));
|
||||
}
|
||||
|
@ -198,7 +209,7 @@ public class CsvInputFormatTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void test_getWeightedSize_withGzCompression()
|
||||
{
|
||||
final CsvInputFormat format = new CsvInputFormat(null, null, true, null, 0);
|
||||
final CsvInputFormat format = new CsvInputFormat(null, null, true, null, 0, null);
|
||||
final long unweightedSize = 100L;
|
||||
Assert.assertEquals(
|
||||
unweightedSize * CompressionUtils.COMPRESSED_TEXT_WEIGHT_FACTOR,
|
||||
|
|
|
@ -61,12 +61,34 @@ public class CsvReaderTest
|
|||
{
|
||||
final ByteEntity source = writeData(
|
||||
ImmutableList.of(
|
||||
"2019-01-01T00:00:10Z,name_1,5",
|
||||
"2019-01-01T00:00:20Z,name_2,10",
|
||||
"2019-01-01T00:00:30Z,name_3,15"
|
||||
"2019-01-01T00:00:10Z,name_1,5,0.0",
|
||||
"2019-01-01T00:00:20Z,name_2,10,1.0",
|
||||
"2019-01-01T00:00:30Z,name_3,15,2.0"
|
||||
)
|
||||
);
|
||||
final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "score"), null, null, false, 0);
|
||||
final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "score"), null, null, false, 0, null);
|
||||
assertResult(source, format);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseNumbers() throws IOException
|
||||
{
|
||||
final ByteEntity source = writeData(
|
||||
ImmutableList.of(
|
||||
"2019-01-01T00:00:10Z,name_1,5,0.0",
|
||||
"2019-01-01T00:00:20Z,name_2,10,1.0",
|
||||
"2019-01-01T00:00:30Z,name_3,15,2.0"
|
||||
)
|
||||
);
|
||||
final CsvInputFormat format = new CsvInputFormat(
|
||||
ImmutableList.of("ts", "name", "score", "rating_dbl"),
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
0,
|
||||
true
|
||||
);
|
||||
|
||||
assertResult(source, format);
|
||||
}
|
||||
|
||||
|
@ -81,7 +103,7 @@ public class CsvReaderTest
|
|||
"2019-01-01T00:00:30Z,name_3,15"
|
||||
)
|
||||
);
|
||||
final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), null, null, true, 0);
|
||||
final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), null, null, true, 0, null);
|
||||
assertResult(source, format);
|
||||
}
|
||||
|
||||
|
@ -96,7 +118,7 @@ public class CsvReaderTest
|
|||
"2019-01-01T00:00:30Z,name_3,15"
|
||||
)
|
||||
);
|
||||
final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "score"), null, null, false, 1);
|
||||
final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "score"), null, null, false, 1, null);
|
||||
assertResult(source, format);
|
||||
}
|
||||
|
||||
|
@ -112,7 +134,7 @@ public class CsvReaderTest
|
|||
"2019-01-01T00:00:30Z,name_3,15"
|
||||
)
|
||||
);
|
||||
final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), null, null, true, 1);
|
||||
final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), null, null, true, 1, null);
|
||||
assertResult(source, format);
|
||||
}
|
||||
|
||||
|
@ -127,7 +149,7 @@ public class CsvReaderTest
|
|||
"2019-01-01T00:00:30Z,name_3,15|3"
|
||||
)
|
||||
);
|
||||
final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), "|", null, true, 0);
|
||||
final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), "|", null, true, 0, null);
|
||||
final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA, source, null);
|
||||
int numResults = 0;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
|
@ -227,7 +249,124 @@ public class CsvReaderTest
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("Timestamp", "auto", null),
|
||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("Timestamp"))),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
source,
|
||||
null
|
||||
);
|
||||
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
final Iterator<InputRow> expectedRowIterator = expectedResults.iterator();
|
||||
while (iterator.hasNext()) {
|
||||
Assert.assertTrue(expectedRowIterator.hasNext());
|
||||
Assert.assertEquals(expectedRowIterator.next(), iterator.next());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDataWithParseNumbersEnabled() throws IOException
|
||||
{
|
||||
final ByteEntity source = writeData(
|
||||
ImmutableList.of(
|
||||
"3,1.0,2,1|2|3,false,2018-05-05T10:00:00Z",
|
||||
"34,-2.0,foo,1,true,2018-05-06T10:00:00Z",
|
||||
"343,3.0,bar,2|3|4,false,2018-05-07T10:00:00Z",
|
||||
"545,-4.0,7,2,true,2018-05-08T10:00:00Z",
|
||||
"65,5.0,baz,3|4,false,2018-05-09T10:00:00Z"
|
||||
)
|
||||
);
|
||||
final RowSignature signature =
|
||||
RowSignature.builder()
|
||||
.add("Value1", null)
|
||||
.add("Value2", null)
|
||||
.add("Value3", null)
|
||||
.add("Value4", null)
|
||||
.add("Value5", null)
|
||||
.add("Timestamp", null)
|
||||
.build();
|
||||
|
||||
final List<InputRow> expectedResults = ImmutableList.of(
|
||||
new ListBasedInputRow(
|
||||
signature,
|
||||
DateTimes.of("2018-05-05T10:00:00Z"),
|
||||
ImmutableList.of("Timestamp"),
|
||||
ImmutableList.of(
|
||||
3L,
|
||||
1.0,
|
||||
2L,
|
||||
ImmutableList.of(1L, 2L, 3L),
|
||||
"false",
|
||||
"2018-05-05T10:00:00Z"
|
||||
)
|
||||
),
|
||||
new ListBasedInputRow(
|
||||
signature,
|
||||
DateTimes.of("2018-05-06T10:00:00Z"),
|
||||
ImmutableList.of("Timestamp"),
|
||||
ImmutableList.of(
|
||||
34L,
|
||||
-2.0,
|
||||
"foo",
|
||||
1L,
|
||||
"true",
|
||||
"2018-05-06T10:00:00Z"
|
||||
)
|
||||
),
|
||||
new ListBasedInputRow(
|
||||
signature,
|
||||
DateTimes.of("2018-05-07T10:00:00Z"),
|
||||
ImmutableList.of("Timestamp"),
|
||||
ImmutableList.of(
|
||||
343L,
|
||||
3.0,
|
||||
"bar",
|
||||
ImmutableList.of(2L, 3L, 4L),
|
||||
"false",
|
||||
"2018-05-07T10:00:00Z"
|
||||
)
|
||||
),
|
||||
new ListBasedInputRow(
|
||||
signature,
|
||||
DateTimes.of("2018-05-08T10:00:00Z"),
|
||||
ImmutableList.of("Timestamp"),
|
||||
ImmutableList.of(
|
||||
545L,
|
||||
-4.0,
|
||||
7L,
|
||||
2L,
|
||||
"true",
|
||||
"2018-05-08T10:00:00Z"
|
||||
)
|
||||
),
|
||||
new ListBasedInputRow(
|
||||
signature,
|
||||
DateTimes.of("2018-05-09T10:00:00Z"),
|
||||
ImmutableList.of("Timestamp"),
|
||||
ImmutableList.of(
|
||||
65L,
|
||||
5.0,
|
||||
"baz",
|
||||
ImmutableList.of(3L, 4L),
|
||||
"false",
|
||||
"2018-05-09T10:00:00Z"
|
||||
)
|
||||
)
|
||||
);
|
||||
final CsvInputFormat format = new CsvInputFormat(
|
||||
ImmutableList.of("Value1", "Value2", "Value3", "Value4", "Value5", "Timestamp"),
|
||||
"|",
|
||||
null,
|
||||
false,
|
||||
0,
|
||||
true
|
||||
);
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
|
@ -256,7 +395,7 @@ public class CsvReaderTest
|
|||
"2019-01-01T00:00:10Z,name_1,\"Как говорится: \\\"\"всё течет, всё изменяется\\\"\". Украина как всегда обвиняет Россию в собственных проблемах. #ПровокацияКиева\""
|
||||
)
|
||||
);
|
||||
final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "Comment"), null, null, false, 0);
|
||||
final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "Comment"), null, null, false, 0, null);
|
||||
final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA, source, null);
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
Assert.assertTrue(iterator.hasNext());
|
||||
|
@ -300,6 +439,17 @@ public class CsvReaderTest
|
|||
StringUtils.format("name_%d", numResults + 1),
|
||||
Iterables.getOnlyElement(row.getDimension("name"))
|
||||
);
|
||||
if (format.shouldTryParseNumbers()) {
|
||||
Assert.assertEquals(((numResults + 1) * 5L), row.getRaw("score"));
|
||||
if (format.getColumns().contains("rating_dbl")) {
|
||||
Assert.assertEquals(numResults * 1.0, row.getRaw("rating_dbl"));
|
||||
}
|
||||
} else {
|
||||
Assert.assertEquals(Integer.toString((numResults + 1) * 5), row.getRaw("score"));
|
||||
if (format.getColumns().contains("rating_dbl")) {
|
||||
Assert.assertEquals(Double.toString(numResults * 1.0), row.getRaw("rating_dbl"));
|
||||
}
|
||||
}
|
||||
Assert.assertEquals(
|
||||
Integer.toString((numResults + 1) * 5),
|
||||
Iterables.getOnlyElement(row.getDimension("score"))
|
||||
|
|
|
@ -45,7 +45,8 @@ public class DelimitedInputFormatTest
|
|||
"delim",
|
||||
null,
|
||||
true,
|
||||
10
|
||||
10,
|
||||
null
|
||||
);
|
||||
final byte[] bytes = mapper.writeValueAsBytes(format);
|
||||
final DelimitedInputFormat fromJson = (DelimitedInputFormat) mapper.readValue(bytes, InputFormat.class);
|
||||
|
@ -57,7 +58,7 @@ public class DelimitedInputFormatTest
|
|||
{
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Column[a\t] cannot have the delimiter[\t] in its name");
|
||||
new DelimitedInputFormat(Collections.singletonList("a\t"), ",", null, null, false, 0);
|
||||
new DelimitedInputFormat(Collections.singletonList("a\t"), ",", null, null, false, 0, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -65,7 +66,7 @@ public class DelimitedInputFormatTest
|
|||
{
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Cannot have same delimiter and list delimiter of [,]");
|
||||
new DelimitedInputFormat(Collections.singletonList("a\t"), ",", ",", null, false, 0);
|
||||
new DelimitedInputFormat(Collections.singletonList("a\t"), ",", ",", null, false, 0, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -73,7 +74,7 @@ public class DelimitedInputFormatTest
|
|||
{
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Column[a|] cannot have the delimiter[|] in its name");
|
||||
new DelimitedInputFormat(Collections.singletonList("a|"), ",", "|", null, false, 0);
|
||||
new DelimitedInputFormat(Collections.singletonList("a|"), ",", "|", null, false, 0, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -85,7 +86,8 @@ public class DelimitedInputFormatTest
|
|||
"delim",
|
||||
null,
|
||||
true,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
Assert.assertTrue(format.isFindColumnsFromHeader());
|
||||
}
|
||||
|
@ -99,17 +101,44 @@ public class DelimitedInputFormatTest
|
|||
"delim",
|
||||
null,
|
||||
true,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
Assert.assertTrue(format.isFindColumnsFromHeader());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTryParseNumbers()
|
||||
{
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(
|
||||
null,
|
||||
null,
|
||||
"delim",
|
||||
null,
|
||||
true,
|
||||
0,
|
||||
true
|
||||
);
|
||||
Assert.assertTrue(format.shouldTryParseNumbers());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeserializeWithTryParseNumbers() throws IOException
|
||||
{
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
final DelimitedInputFormat inputFormat = (DelimitedInputFormat) mapper.readValue(
|
||||
"{\"type\":\"tsv\",\"hasHeaderRow\":true,\"tryParseNumbers\":true}",
|
||||
InputFormat.class
|
||||
);
|
||||
Assert.assertTrue(inputFormat.shouldTryParseNumbers());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMissingFindColumnsFromHeaderWithMissingColumnsThrowingError()
|
||||
{
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Either [columns] or [findColumnsFromHeader] must be set");
|
||||
new DelimitedInputFormat(null, null, "delim", null, null, 0);
|
||||
new DelimitedInputFormat(null, null, "delim", null, null, 0, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -121,7 +150,8 @@ public class DelimitedInputFormatTest
|
|||
"delim",
|
||||
null,
|
||||
null,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
Assert.assertFalse(format.isFindColumnsFromHeader());
|
||||
}
|
||||
|
@ -131,19 +161,19 @@ public class DelimitedInputFormatTest
|
|||
{
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Cannot accept both [findColumnsFromHeader] and [hasHeaderRow]");
|
||||
new DelimitedInputFormat(null, null, "delim", true, false, 0);
|
||||
new DelimitedInputFormat(null, null, "delim", true, false, 0, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHasHeaderRowWithMissingColumnsReturningItsValue()
|
||||
{
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(null, null, "delim", true, null, 0);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(null, null, "delim", true, null, 0, null);
|
||||
Assert.assertTrue(format.isFindColumnsFromHeader());
|
||||
}
|
||||
@Test
|
||||
public void test_getWeightedSize_withoutCompression()
|
||||
{
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(null, null, "delim", true, null, 0);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(null, null, "delim", true, null, 0, null);
|
||||
final long unweightedSize = 100L;
|
||||
Assert.assertEquals(unweightedSize, format.getWeightedSize("file.tsv", unweightedSize));
|
||||
}
|
||||
|
@ -151,7 +181,7 @@ public class DelimitedInputFormatTest
|
|||
@Test
|
||||
public void test_getWeightedSize_withGzCompression()
|
||||
{
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(null, null, "delim", true, null, 0);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(null, null, "delim", true, null, 0, null);
|
||||
final long unweightedSize = 100L;
|
||||
Assert.assertEquals(
|
||||
unweightedSize * CompressionUtils.COMPRESSED_TEXT_WEIGHT_FACTOR,
|
||||
|
|
|
@ -26,9 +26,11 @@ import org.apache.druid.data.input.ColumnsFilter;
|
|||
import org.apache.druid.data.input.InputEntityReader;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.InputRowSchema;
|
||||
import org.apache.druid.data.input.ListBasedInputRow;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.parsers.CloseableIterator;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
@ -36,6 +38,7 @@ import org.junit.Test;
|
|||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
@ -58,18 +61,41 @@ public class DelimitedReaderTest
|
|||
{
|
||||
final ByteEntity source = writeData(
|
||||
ImmutableList.of(
|
||||
"2019-01-01T00:00:10Z\tname_1\t5",
|
||||
"2019-01-01T00:00:20Z\tname_2\t10",
|
||||
"2019-01-01T00:00:30Z\tname_3\t15"
|
||||
"2019-01-01T00:00:10Z\tname_1\t5\t0.0",
|
||||
"2019-01-01T00:00:20Z\tname_2\t10\t1.0",
|
||||
"2019-01-01T00:00:30Z\tname_3\t15\t2.0"
|
||||
)
|
||||
);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(
|
||||
ImmutableList.of("ts", "name", "score"),
|
||||
ImmutableList.of("ts", "name", "score", "rating_dbl"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
assertResult(source, format);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseNumbers() throws IOException
|
||||
{
|
||||
final ByteEntity source = writeData(
|
||||
ImmutableList.of(
|
||||
"2019-01-01T00:00:10Z\tname_1\t5\t0.0",
|
||||
"2019-01-01T00:00:20Z\tname_2\t10\t1.0",
|
||||
"2019-01-01T00:00:30Z\tname_3\t15\t2.0"
|
||||
)
|
||||
);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(
|
||||
ImmutableList.of("ts", "name", "score", "rating_dbl"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
0,
|
||||
true
|
||||
);
|
||||
assertResult(source, format);
|
||||
}
|
||||
|
@ -85,7 +111,7 @@ public class DelimitedReaderTest
|
|||
"2019-01-01T00:00:30Z\tname_3\t15"
|
||||
)
|
||||
);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), null, null, null, true, 0);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), null, null, null, true, 0, null);
|
||||
assertResult(source, format);
|
||||
}
|
||||
|
||||
|
@ -106,7 +132,8 @@ public class DelimitedReaderTest
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
1
|
||||
1,
|
||||
null
|
||||
);
|
||||
assertResult(source, format);
|
||||
}
|
||||
|
@ -123,7 +150,7 @@ public class DelimitedReaderTest
|
|||
"2019-01-01T00:00:30Z\tname_3\t15"
|
||||
)
|
||||
);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), null, null, null, true, 1);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), null, null, null, true, 1, null);
|
||||
assertResult(source, format);
|
||||
}
|
||||
|
||||
|
@ -138,7 +165,7 @@ public class DelimitedReaderTest
|
|||
"2019-01-01T00:00:30Z\tname_3\t15|3"
|
||||
)
|
||||
);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), "|", null, null, true, 0);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), "|", null, null, true, 0, null);
|
||||
final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA, source, null);
|
||||
int numResults = 0;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
|
@ -162,6 +189,123 @@ public class DelimitedReaderTest
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDataWithParseNumbersEnabled() throws IOException
|
||||
{
|
||||
final ByteEntity source = writeData(
|
||||
ImmutableList.of(
|
||||
"3\t1.0\t2\t1|2|3\tfalse\t2018-05-05T10:00:00Z",
|
||||
"34\t-2.0\tfoo\t1\ttrue\t2018-05-06T10:00:00Z",
|
||||
"343\t3.0\tbar\t2|3|4\tfalse\t2018-05-07T10:00:00Z",
|
||||
"545\t-4.0\t7\t2\ttrue\t2018-05-08T10:00:00Z",
|
||||
"65\t5.0\tbaz\t3|4\tfalse\t2018-05-09T10:00:00Z"
|
||||
)
|
||||
);
|
||||
final RowSignature signature =
|
||||
RowSignature.builder()
|
||||
.add("Value1", null)
|
||||
.add("Value2", null)
|
||||
.add("Value3", null)
|
||||
.add("Value4", null)
|
||||
.add("Value5", null)
|
||||
.add("Timestamp", null)
|
||||
.build();
|
||||
|
||||
final List<InputRow> expectedResults = ImmutableList.of(
|
||||
new ListBasedInputRow(
|
||||
signature,
|
||||
DateTimes.of("2018-05-05T10:00:00Z"),
|
||||
ImmutableList.of("Timestamp"),
|
||||
ImmutableList.of(
|
||||
3L,
|
||||
1.0,
|
||||
2L,
|
||||
ImmutableList.of(1L, 2L, 3L),
|
||||
"false",
|
||||
"2018-05-05T10:00:00Z"
|
||||
)
|
||||
),
|
||||
new ListBasedInputRow(
|
||||
signature,
|
||||
DateTimes.of("2018-05-06T10:00:00Z"),
|
||||
ImmutableList.of("Timestamp"),
|
||||
ImmutableList.of(
|
||||
34L,
|
||||
-2.0,
|
||||
"foo",
|
||||
1L,
|
||||
"true",
|
||||
"2018-05-06T10:00:00Z"
|
||||
)
|
||||
),
|
||||
new ListBasedInputRow(
|
||||
signature,
|
||||
DateTimes.of("2018-05-07T10:00:00Z"),
|
||||
ImmutableList.of("Timestamp"),
|
||||
ImmutableList.of(
|
||||
343L,
|
||||
3.0,
|
||||
"bar",
|
||||
ImmutableList.of(2L, 3L, 4L),
|
||||
"false",
|
||||
"2018-05-07T10:00:00Z"
|
||||
)
|
||||
),
|
||||
new ListBasedInputRow(
|
||||
signature,
|
||||
DateTimes.of("2018-05-08T10:00:00Z"),
|
||||
ImmutableList.of("Timestamp"),
|
||||
ImmutableList.of(
|
||||
545L,
|
||||
-4.0,
|
||||
7L,
|
||||
2L,
|
||||
"true",
|
||||
"2018-05-08T10:00:00Z"
|
||||
)
|
||||
),
|
||||
new ListBasedInputRow(
|
||||
signature,
|
||||
DateTimes.of("2018-05-09T10:00:00Z"),
|
||||
ImmutableList.of("Timestamp"),
|
||||
ImmutableList.of(
|
||||
65L,
|
||||
5.0,
|
||||
"baz",
|
||||
ImmutableList.of(3L, 4L),
|
||||
"false",
|
||||
"2018-05-09T10:00:00Z"
|
||||
)
|
||||
)
|
||||
);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(
|
||||
ImmutableList.of("Value1", "Value2", "Value3", "Value4", "Value5", "Timestamp"),
|
||||
"|",
|
||||
null,
|
||||
false,
|
||||
null,
|
||||
0,
|
||||
true
|
||||
);
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("Timestamp", "auto", null),
|
||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("Timestamp"))),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
source,
|
||||
null
|
||||
);
|
||||
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
final Iterator<InputRow> expectedRowIterator = expectedResults.iterator();
|
||||
while (iterator.hasNext()) {
|
||||
Assert.assertTrue(expectedRowIterator.hasNext());
|
||||
Assert.assertEquals(expectedRowIterator.next(), iterator.next());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCustomizeSeparator() throws IOException
|
||||
{
|
||||
|
@ -173,7 +317,7 @@ public class DelimitedReaderTest
|
|||
"2019-01-01T00:00:30Z|name_3|15\t3"
|
||||
)
|
||||
);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), "\t", "|", null, true, 0);
|
||||
final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), "\t", "|", null, true, 0, null);
|
||||
final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA, source, null);
|
||||
int numResults = 0;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
|
@ -211,7 +355,8 @@ public class DelimitedReaderTest
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA, source, null);
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
|
@ -256,6 +401,17 @@ public class DelimitedReaderTest
|
|||
StringUtils.format("name_%d", numResults + 1),
|
||||
Iterables.getOnlyElement(row.getDimension("name"))
|
||||
);
|
||||
if (format.shouldTryParseNumbers()) {
|
||||
Assert.assertEquals(((numResults + 1) * 5L), row.getRaw("score"));
|
||||
if (format.getColumns().contains("rating_dbl")) {
|
||||
Assert.assertEquals(numResults * 1.0, row.getRaw("rating_dbl"));
|
||||
}
|
||||
} else {
|
||||
Assert.assertEquals(Integer.toString((numResults + 1) * 5), row.getRaw("score"));
|
||||
if (format.getColumns().contains("rating_dbl")) {
|
||||
Assert.assertEquals(Double.toString(numResults * 1.0), row.getRaw("rating_dbl"));
|
||||
}
|
||||
}
|
||||
Assert.assertEquals(
|
||||
Integer.toString((numResults + 1) * 5),
|
||||
Iterables.getOnlyElement(row.getDimension("score"))
|
||||
|
|
|
@ -84,7 +84,8 @@ public class InputEntityIteratingReaderTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
),
|
||||
CloseableIterators.withEmptyBaggage(
|
||||
files.stream().flatMap(file -> ImmutableList.of(new FileEntity(file)).stream()).iterator()
|
||||
|
@ -152,7 +153,8 @@ public class InputEntityIteratingReaderTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
),
|
||||
CloseableIterators.withEmptyBaggage(
|
||||
files.stream().flatMap(file -> ImmutableList.of(new FileEntity(file)).stream()).iterator()
|
||||
|
@ -200,7 +202,8 @@ public class InputEntityIteratingReaderTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
),
|
||||
CloseableIterators.withEmptyBaggage(
|
||||
ImmutableList.of(
|
||||
|
|
|
@ -112,7 +112,7 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest
|
|||
public void testWithSkipHeaderRows()
|
||||
{
|
||||
final int skipHeaderRows = 2;
|
||||
final Parser<String, Object> parser = PARSER_FACTORY.get(format, false, skipHeaderRows);
|
||||
final Parser<String, Object> parser = PARSER_FACTORY.get(format, false, skipHeaderRows, false);
|
||||
parser.startFileFromBeginning();
|
||||
final String[] body = new String[]{
|
||||
concat(format, "header", "line", "1"),
|
||||
|
@ -134,7 +134,7 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testWithHeaderRow()
|
||||
{
|
||||
final Parser<String, Object> parser = PARSER_FACTORY.get(format, true, 0);
|
||||
final Parser<String, Object> parser = PARSER_FACTORY.get(format, true, 0, false);
|
||||
parser.startFileFromBeginning();
|
||||
final String[] body = new String[]{
|
||||
concat(format, "time", "value1", "value2"),
|
||||
|
@ -152,7 +152,7 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testWithHeaderRowOfEmptyColumns()
|
||||
{
|
||||
final Parser<String, Object> parser = PARSER_FACTORY.get(format, true, 0);
|
||||
final Parser<String, Object> parser = PARSER_FACTORY.get(format, true, 0, false);
|
||||
parser.startFileFromBeginning();
|
||||
final String[] body = new String[]{
|
||||
concat(format, "time", "", "value2", ""),
|
||||
|
@ -170,7 +170,7 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testWithDifferentHeaderRows()
|
||||
{
|
||||
final Parser<String, Object> parser = PARSER_FACTORY.get(format, true, 0);
|
||||
final Parser<String, Object> parser = PARSER_FACTORY.get(format, true, 0, false);
|
||||
parser.startFileFromBeginning();
|
||||
final String[] body = new String[]{
|
||||
concat(format, "time", "value1", "value2"),
|
||||
|
@ -207,7 +207,7 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest
|
|||
);
|
||||
|
||||
final int skipHeaderRows = 2;
|
||||
final Parser<String, Object> parser = PARSER_FACTORY.get(format, false, skipHeaderRows);
|
||||
final Parser<String, Object> parser = PARSER_FACTORY.get(format, false, skipHeaderRows, false);
|
||||
final String[] body = new String[]{
|
||||
concat(format, "header", "line", "1"),
|
||||
concat(format, "header", "line", "2"),
|
||||
|
@ -219,7 +219,7 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testWithNullValues()
|
||||
{
|
||||
final Parser<String, Object> parser = PARSER_FACTORY.get(format, true, 0);
|
||||
final Parser<String, Object> parser = PARSER_FACTORY.get(format, true, 0, false);
|
||||
parser.startFileFromBeginning();
|
||||
final String[] body = new String[]{
|
||||
concat(format, "time", "value1", "value2"),
|
||||
|
@ -234,16 +234,16 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest
|
|||
{
|
||||
public Parser<String, Object> get(FlatTextFormat format)
|
||||
{
|
||||
return get(format, false, 0);
|
||||
return get(format, false, 0, false);
|
||||
}
|
||||
|
||||
public Parser<String, Object> get(FlatTextFormat format, boolean hasHeaderRow, int maxSkipHeaderRows)
|
||||
public Parser<String, Object> get(FlatTextFormat format, boolean hasHeaderRow, int maxSkipHeaderRows, boolean tryParseNumbers)
|
||||
{
|
||||
switch (format) {
|
||||
case CSV:
|
||||
return new CSVParser(null, hasHeaderRow, maxSkipHeaderRows);
|
||||
return new CSVParser(null, hasHeaderRow, maxSkipHeaderRows, tryParseNumbers);
|
||||
case DELIMITED:
|
||||
return new DelimitedParser("\t", null, hasHeaderRow, maxSkipHeaderRows);
|
||||
return new DelimitedParser("\t", null, hasHeaderRow, maxSkipHeaderRows, tryParseNumbers);
|
||||
default:
|
||||
throw new IAE("Unknown format[%s]", format);
|
||||
}
|
||||
|
|
|
@ -19,27 +19,269 @@
|
|||
|
||||
package org.apache.druid.java.util.common.parsers;
|
||||
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.junit.Assert;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import static org.apache.druid.java.util.common.parsers.ParserUtils.findDuplicates;
|
||||
import static org.apache.druid.java.util.common.parsers.ParserUtils.getTransformationFunction;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
|
||||
public class ParserUtilsTest
|
||||
{
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFindDuplicatesMixedCases()
|
||||
{
|
||||
final List<String> fields = ImmutableList.of("f1", "f2", "F1", "F2", "f3");
|
||||
Assert.assertEquals(Collections.emptySet(), ParserUtils.findDuplicates(fields));
|
||||
assertEquals(Collections.emptySet(), findDuplicates(fields));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFindDuplicates()
|
||||
{
|
||||
final List<String> fields = ImmutableList.of("f1", "f2", "F1", "F2", "f1", "F2");
|
||||
Assert.assertEquals(ImmutableSet.of("f1", "F2"), ParserUtils.findDuplicates(fields));
|
||||
assertEquals(ImmutableSet.of("f1", "F2"), findDuplicates(fields));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInputWithDelimiterAndParserDisabled()
|
||||
{
|
||||
assertNull(
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply(null)
|
||||
);
|
||||
assertEquals(
|
||||
NullHandling.emptyToNullIfNeeded(""),
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply("")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of("foo", "boo"),
|
||||
getTransformationFunction("|", Splitter.on("|"), false).apply("foo|boo")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of("1", "2", "3"),
|
||||
getTransformationFunction("|", Splitter.on("|"), false).apply("1|2|3")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of("1", "-2", "3", "0", "-2"),
|
||||
getTransformationFunction("|", Splitter.on("|"), false).apply("1|-2|3|0|-2")
|
||||
);
|
||||
assertEquals(
|
||||
"100",
|
||||
getTransformationFunction("|", Splitter.on("|"), false).apply("100")
|
||||
);
|
||||
assertEquals(
|
||||
"1.23",
|
||||
getTransformationFunction("|", Splitter.on("|"), false).apply("1.23")
|
||||
);
|
||||
assertEquals(
|
||||
"-2.0",
|
||||
getTransformationFunction("|", Splitter.on("|"), false).apply("-2.0")
|
||||
);
|
||||
assertEquals(
|
||||
"1e2",
|
||||
getTransformationFunction("|", Splitter.on("|"), false).apply("1e2")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of("1", "2", "3"),
|
||||
getTransformationFunction("|", Splitter.on("|"), false).apply("1|2|3")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of("1", "-2", "3", "0", "-2"),
|
||||
getTransformationFunction("|", Splitter.on("|"), false).apply("1|-2|3|0|-2")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of("-1.0", "-2.2", "3.1", "0.2", "-2.1"),
|
||||
getTransformationFunction("|", Splitter.on("|"), false).apply("-1.0|-2.2|3.1|0.2|-2.1")
|
||||
);
|
||||
|
||||
// Some mixed types
|
||||
assertEquals(
|
||||
ImmutableList.of("-1.23", "3.13", "23"),
|
||||
getTransformationFunction("|", Splitter.on("|"), false).apply("-1.23|3.13|23")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of("-1.23", "3.13", "23", "foo", "-9"),
|
||||
getTransformationFunction("|", Splitter.on("|"), false).apply("-1.23|3.13|23|foo|-9")
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInputWithDelimiterAndParserEnabled()
|
||||
{
|
||||
assertNull(
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply(null)
|
||||
);
|
||||
assertEquals(
|
||||
NullHandling.emptyToNullIfNeeded(""),
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply("")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of("foo", "boo"),
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply("foo|boo")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of(1L, 2L, 3L),
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply("1|2|3")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of(1L, -2L, 3L, 0L, -2L),
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply("1|-2|3|0|-2")
|
||||
);
|
||||
assertEquals(
|
||||
100L,
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply("100")
|
||||
);
|
||||
assertEquals(
|
||||
1.23,
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply("1.23")
|
||||
);
|
||||
assertEquals(
|
||||
-2.0,
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply("-2.0")
|
||||
);
|
||||
assertEquals(
|
||||
100.0,
|
||||
getTransformationFunction("$", Splitter.on("|"), true).apply("1e2")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of(1L, 2L, 3L),
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply("1|2|3")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of(1L, -2L, 3L, 0L, -2L),
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply("1|-2|3|0|-2")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of(-1.0, -2.2, 3.1, 0.2, -2.1),
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply("-1.0|-2.2|3.1|0.2|-2.1")
|
||||
);
|
||||
|
||||
// Some mixed types
|
||||
assertEquals(
|
||||
ImmutableList.of(-1.23, 3.13, 23L),
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply("-1.23|3.13|23")
|
||||
);
|
||||
assertEquals(
|
||||
ImmutableList.of(-1.23, 3.13, 23L, "foo", -9L),
|
||||
getTransformationFunction("|", Splitter.on("|"), true).apply("-1.23|3.13|23|foo|-9")
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInputWithoutDelimiterAndNumberParsingDisabled()
|
||||
{
|
||||
assertNull(
|
||||
getTransformationFunction("|", Splitter.on("$"), false).apply(null)
|
||||
);
|
||||
assertEquals(
|
||||
NullHandling.emptyToNullIfNeeded(""),
|
||||
getTransformationFunction("|", Splitter.on("$"), false).apply("")
|
||||
);
|
||||
assertEquals(
|
||||
"foo|boo",
|
||||
getTransformationFunction("$", Splitter.on("$"), false).apply("foo|boo")
|
||||
);
|
||||
assertEquals(
|
||||
"100",
|
||||
getTransformationFunction("$", Splitter.on("$"), false).apply("100")
|
||||
);
|
||||
assertEquals(
|
||||
"1.23",
|
||||
getTransformationFunction("$", Splitter.on("$"), false).apply("1.23")
|
||||
);
|
||||
assertEquals(
|
||||
"-2.0",
|
||||
getTransformationFunction("$", Splitter.on("$"), false).apply("-2.0")
|
||||
);
|
||||
assertEquals(
|
||||
"1e2",
|
||||
getTransformationFunction("$", Splitter.on("$"), false).apply("1e2")
|
||||
);
|
||||
assertEquals(
|
||||
"1|2|3",
|
||||
getTransformationFunction("$", Splitter.on("$"), false).apply("1|2|3")
|
||||
);
|
||||
assertEquals(
|
||||
"1|-2|3|0|-2",
|
||||
getTransformationFunction("$", Splitter.on("$"), false).apply("1|-2|3|0|-2")
|
||||
);
|
||||
assertEquals(
|
||||
"-1.0|-2.2|3.1|0.2|-2.1",
|
||||
getTransformationFunction("$", Splitter.on("$"), false).apply("-1.0|-2.2|3.1|0.2|-2.1")
|
||||
);
|
||||
|
||||
// Some mixed types
|
||||
assertEquals(
|
||||
"-1.23|3.13|23",
|
||||
getTransformationFunction("$", Splitter.on("$"), false).apply("-1.23|3.13|23")
|
||||
);
|
||||
assertEquals(
|
||||
"-1.23|3.13|23|foo|-9",
|
||||
getTransformationFunction("$", Splitter.on("$"), false).apply("-1.23|3.13|23|foo|-9")
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInputWithoutDelimiterAndNumberParsingEnabled()
|
||||
{
|
||||
assertNull(
|
||||
getTransformationFunction("$", Splitter.on("$"), true).apply(null)
|
||||
);
|
||||
assertEquals(
|
||||
NullHandling.emptyToNullIfNeeded(""),
|
||||
getTransformationFunction("$", Splitter.on("$"), true).apply("")
|
||||
);
|
||||
assertEquals(
|
||||
"foo|boo",
|
||||
getTransformationFunction("$", Splitter.on("$"), true).apply("foo|boo")
|
||||
);
|
||||
assertEquals(
|
||||
100L,
|
||||
getTransformationFunction("$", Splitter.on("$"), true).apply("100")
|
||||
);
|
||||
assertEquals(
|
||||
1.23,
|
||||
getTransformationFunction("$", Splitter.on("$"), true).apply("1.23")
|
||||
);
|
||||
assertEquals(
|
||||
-2.0,
|
||||
getTransformationFunction("$", Splitter.on("$"), true).apply("-2.0")
|
||||
);
|
||||
assertEquals(
|
||||
100.0,
|
||||
getTransformationFunction("$", Splitter.on("$"), true).apply("1e2")
|
||||
);
|
||||
assertEquals(
|
||||
"1|2|3",
|
||||
getTransformationFunction("$", Splitter.on("$"), true).apply("1|2|3")
|
||||
);
|
||||
assertEquals(
|
||||
"1|-2|3|0|-2",
|
||||
getTransformationFunction("$", Splitter.on("$"), true).apply("1|-2|3|0|-2")
|
||||
);
|
||||
assertEquals(
|
||||
"-1.0|-2.2|3.1|0.2|-2.1",
|
||||
getTransformationFunction("$", Splitter.on("$"), true).apply("-1.0|-2.2|3.1|0.2|-2.1")
|
||||
);
|
||||
|
||||
// Some mixed types
|
||||
assertEquals(
|
||||
"-1.23|3.13|23",
|
||||
getTransformationFunction("$", Splitter.on("$"), true).apply("-1.23|3.13|23")
|
||||
);
|
||||
assertEquals(
|
||||
"-1.23|3.13|23|foo|-9",
|
||||
getTransformationFunction("$", Splitter.on("$"), true).apply("-1.23|3.13|23|foo|-9")
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -147,7 +147,8 @@ public class NestedDataTestUtils
|
|||
null,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
0,
|
||||
null
|
||||
);
|
||||
|
||||
public static final TransformSpec SIMPLE_DATA_TSV_TRANSFORM = new TransformSpec(
|
||||
|
|
|
@ -66,7 +66,7 @@ public class CsvInputFormatTest extends BaseExternTableTest
|
|||
public void testConversion()
|
||||
{
|
||||
CsvInputFormat format = new CsvInputFormat(
|
||||
Collections.singletonList("a"), ";", false, false, 1);
|
||||
Collections.singletonList("a"), ";", false, false, 1, null);
|
||||
TableMetadata table = TableBuilder.external("foo")
|
||||
.inputSource(toMap(new InlineInputSource("a\n")))
|
||||
.inputFormat(formatToMap(format))
|
||||
|
|
|
@ -74,7 +74,7 @@ public class DelimitedInputFormatTest extends BaseExternTableTest
|
|||
public void testConversion()
|
||||
{
|
||||
DelimitedInputFormat format = new DelimitedInputFormat(
|
||||
Collections.singletonList("a"), ";", "|", false, false, 1);
|
||||
Collections.singletonList("a"), ";", "|", false, false, 1, null);
|
||||
TableMetadata table = TableBuilder.external("foo")
|
||||
.inputSource(toMap(new InlineInputSource("a\n")))
|
||||
.inputFormat(formatToMap(format))
|
||||
|
|
|
@ -122,7 +122,7 @@ public class ExternalTableTest extends BaseExternTableTest
|
|||
{
|
||||
// Format is given without columns: it is validated
|
||||
CsvInputFormat format = new CsvInputFormat(
|
||||
Collections.singletonList("a"), ";", false, false, 0);
|
||||
Collections.singletonList("a"), ";", false, false, 0, null);
|
||||
TableMetadata table = TableBuilder.external("foo")
|
||||
.inputSource(toMap(new InlineInputSource("a\n")))
|
||||
.inputFormat(formatToMap(format))
|
||||
|
|
|
@ -194,7 +194,7 @@ public class InlineInputSourceDefnTest extends BaseExternTableTest
|
|||
{
|
||||
// Define an inline table
|
||||
CsvInputFormat format = new CsvInputFormat(
|
||||
Collections.singletonList("a"), ";", false, false, 0);
|
||||
Collections.singletonList("a"), ";", false, false, 0, null);
|
||||
TableMetadata table = TableBuilder.external("foo")
|
||||
.inputSource(toMap(new InlineInputSource("a,b\nc,d")))
|
||||
.inputFormat(formatToMap(format))
|
||||
|
|
|
@ -479,7 +479,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml
|
|||
{
|
||||
ExternalDataSource externalDataSource = new ExternalDataSource(
|
||||
new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("a", ColumnType.STRING)
|
||||
.add("b", ColumnType.STRING)
|
||||
|
@ -541,7 +541,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml
|
|||
{
|
||||
ExternalDataSource externalDataSource = new ExternalDataSource(
|
||||
new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("a", ColumnType.STRING)
|
||||
.add("b", ColumnType.STRING)
|
||||
|
@ -603,7 +603,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml
|
|||
{
|
||||
ExternalDataSource externalDataSource = new ExternalDataSource(
|
||||
new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("a", ColumnType.STRING)
|
||||
.add("b", ColumnType.STRING)
|
||||
|
@ -665,7 +665,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml
|
|||
{
|
||||
ExternalDataSource externalDataSource = new ExternalDataSource(
|
||||
new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("a", ColumnType.STRING)
|
||||
.add("b", ColumnType.STRING)
|
||||
|
@ -834,7 +834,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml
|
|||
{
|
||||
ExternalDataSource externalDataSource = new ExternalDataSource(
|
||||
new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("a", ColumnType.STRING)
|
||||
.add("b", ColumnType.STRING)
|
||||
|
@ -953,7 +953,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml
|
|||
{
|
||||
ExternalDataSource externalDataSource = new ExternalDataSource(
|
||||
new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("a", ColumnType.STRING)
|
||||
.add("b", ColumnType.STRING)
|
||||
|
@ -1012,7 +1012,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml
|
|||
{
|
||||
ExternalDataSource externalDataSource = new ExternalDataSource(
|
||||
new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("a", ColumnType.STRING)
|
||||
.add("b", ColumnType.STRING)
|
||||
|
@ -1077,7 +1077,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml
|
|||
{
|
||||
ExternalDataSource externalDataSource = new ExternalDataSource(
|
||||
new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("a", ColumnType.STRING)
|
||||
.add("b", ColumnType.STRING)
|
||||
|
|
|
@ -116,7 +116,7 @@ public class CalciteIngestionDmlTest extends BaseCalciteQueryTest
|
|||
|
||||
protected final ExternalDataSource externalDataSource = new ExternalDataSource(
|
||||
new InlineInputSource("a,b,1\nc,d,2\n"),
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("x", ColumnType.STRING)
|
||||
.add("y", ColumnType.STRING)
|
||||
|
|
|
@ -369,7 +369,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
|
|||
),
|
||||
Calcites.escapeStringLiteral(
|
||||
queryJsonMapper.writeValueAsString(
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0)
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
@ -414,7 +414,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
|
|||
),
|
||||
Calcites.escapeStringLiteral(
|
||||
queryJsonMapper.writeValueAsString(
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0)
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
@ -458,7 +458,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
|
|||
new InlineInputSource("a,b,1\nc,d,2\n")
|
||||
),
|
||||
queryJsonMapper.writeValueAsString(
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0)
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -494,7 +494,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
|
|||
new TestFileInputSource(ImmutableList.of(new File("/tmp/foo.csv").getAbsoluteFile()));
|
||||
final ExternalDataSource externalDataSource = new ExternalDataSource(
|
||||
inputSource,
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("x", ColumnType.STRING)
|
||||
.add("y", ColumnType.STRING)
|
||||
|
@ -510,7 +510,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
|
|||
+ "inputFormat => '%s'))",
|
||||
queryJsonMapper.writeValueAsString(inputSource),
|
||||
queryJsonMapper.writeValueAsString(
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0)
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -552,7 +552,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
|
|||
queryJsonMapper.writeValueAsString(
|
||||
new TestFileInputSource(ImmutableList.of(new File("/tmp/foo.csv").getAbsoluteFile()))),
|
||||
queryJsonMapper.writeValueAsString(
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0)
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -1488,7 +1488,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
|
|||
{
|
||||
ExternalDataSource restrictedSignature = new ExternalDataSource(
|
||||
new InlineInputSource("100\nc200\n"),
|
||||
new CsvInputFormat(ImmutableList.of("__time"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("__time"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("__time", ColumnType.STRING)
|
||||
.build()
|
||||
|
|
|
@ -106,7 +106,7 @@ public class IngestTableFunctionTest extends CalciteIngestionDmlTest
|
|||
null,
|
||||
new HttpInputSourceConfig(null, null)
|
||||
),
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("x", ColumnType.STRING)
|
||||
.add("y", ColumnType.STRING)
|
||||
|
@ -120,7 +120,7 @@ public class IngestTableFunctionTest extends CalciteIngestionDmlTest
|
|||
Arrays.asList(new File("/tmp/foo.csv"), new File("/tmp/bar.csv")),
|
||||
SystemFields.none()
|
||||
),
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("x", ColumnType.STRING)
|
||||
.add("y", ColumnType.STRING)
|
||||
|
@ -294,7 +294,7 @@ public class IngestTableFunctionTest extends CalciteIngestionDmlTest
|
|||
ImmutableMap.of("Accept", "application/ndjson", "a", "b"),
|
||||
new HttpInputSourceConfig(null, Sets.newHashSet("Accept", "a"))
|
||||
),
|
||||
new CsvInputFormat(ImmutableList.of("timestamp", "isRobot"), null, false, false, 0),
|
||||
new CsvInputFormat(ImmutableList.of("timestamp", "isRobot"), null, false, false, 0, null),
|
||||
RowSignature.builder()
|
||||
.add("timestamp", ColumnType.STRING)
|
||||
.add("isRobot", ColumnType.STRING)
|
||||
|
|
|
@ -242,6 +242,7 @@ TLS
|
|||
tls
|
||||
TopN
|
||||
TopNs
|
||||
tryParseNumbers
|
||||
UI
|
||||
UIs
|
||||
UPSERT
|
||||
|
|
Loading…
Reference in New Issue