Skip to content

Fix ingestion failure of pretty-formatted JSON message#10383

Merged
jihoonson merged 26 commits intoapache:masterfrom
FrankChen021:json_parse_bug
Nov 13, 2020
Merged

Fix ingestion failure of pretty-formatted JSON message#10383
jihoonson merged 26 commits intoapache:masterfrom
FrankChen021:json_parse_bug

Conversation

@FrankChen021
Copy link
Copy Markdown
Member

@FrankChen021 FrankChen021 commented Sep 11, 2020

This PR fixes #10259 .

Description

The cause and solution has been discussed completely in the issue.

In this PR,

  1. a new JsonReader is provided to read json text for streaming ingestion
  2. original JsonReader is renamed to JsonLineReader for batch ingestion

This PR has:

  • been self-reviewed.
  • using the concurrency checklist (Remove this item if the PR doesn't have any relation to concurrency.)
  • added documentation for new or modified features or behaviors.
  • added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
  • added or updated version, license, or notice information in licenses.yaml
  • added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
  • added unit tests or modified existing tests to cover new code paths, ensuring the threshold for code coverage is met.
  • added integration tests.
  • been tested in a test Druid cluster.

@FrankChen021
Copy link
Copy Markdown
Member Author

CI reports some failures of integration tests, I'll check it later

@FrankChen021
Copy link
Copy Markdown
Member Author

FrankChen021 commented Sep 11, 2020

Once a json text line is ill-formed, the next call on iterator returned by ObjectMapper.readValues throws exception, and it seems we have no opportunity to process the rest lines. This is why some old test cases fail.

I don't know whether jackson provides anyway to skip the ill-formed text, but I guess there's no such API for a parser.
So what's your suggestion on this problem ? @gianm @jihoonson

@suneet-s
Copy link
Copy Markdown
Contributor

@FrankChen021 I think we should add integration tests for this. Consider updating the data files in integration-tests/src/test/resources/data/batch_index/json/ to include multi-line json

The CI failures look legit. I re-triggered the failing integration test since it looked like it might have been flaky.

@gianm
Copy link
Copy Markdown
Contributor

gianm commented Sep 11, 2020

I don't know whether jackson provides anyway to skip the ill-formed text, but I guess there's no such API for a parser.
So what's your suggestion on this problem ? @gianm @jihoonson

Hmm, this is definitely a problem. When we're reading JSON from a file, we should be skipping the "lines" that aren't parseable, and marking them as unparseable. I guess that ObjectMapper API was too good to be true 🙂

Maybe we can do one of the following two things.

Option 1: After an error we can manually skip to the line that starts with { and begin parsing there. Because the JsonParser has a buffer, we need to make sure we don't miss buffered but unparsed content. We might be able to do that with the JsonParser.releaseBuffered method, assuming it works properly after an error.

Option 2: Introduce a lineSplittable parameter for the JsonReader like you originally suggested, but it would behave differently in two ways. First: it wouldn't be available on the JsonInputFormat. Instead, it would be automatically set to true for batch ingestion and false for streaming ingestion. Second: false doesn't mean "not line splittable", it just means "we should auto detect". In the true case (batch ingestion), the JsonReader will split on lines and parse each line individually. Batch ingestion won't support pretty-printed input, but it never did in the past, and it doesn't seem likely that it's a common requirement anyway. In the false case (streaming ingestion), the JsonReader would use the ObjectMapper.readValues API, and it would support all these different kinds of payloads. But if there's an error, it would reject the entire payload. That all-or-nothing behavior seems OK for a streaming ingestion use case.

IMO Option 2 sounds nicer, since it doesn't involve doing weird stuff with the ObjectReader APIs to try to skip unparseable data.

@jihoonson
Copy link
Copy Markdown
Contributor

@FrankChen021 good catch. The option 2 among @gianm's suggestions sounds better to me. The option 1 sounds hard to actually implement since there will be lots of edge cases we should handle.

Regarding integration tests, I think it would depend on what this PR is going to change. If it needs to modify some behavior in interaction between supervisor and tasks (for example, if the supervisor should set the lineSplittable flag properly), we need some. If the change can be contained only on the task side, I think adding some new unit tests in KafkaIndexTask would be enough. @suneet-s what do you think?

@FrankChen021
Copy link
Copy Markdown
Member Author

FrankChen021 commented Sep 12, 2020

@gianm @jihoonson Thanks for the suggestions. Option 2 is much easier for us to control. I would like to do it as the following.

As parameter lineSplittable is introduced on JsonInputFormat, to make the code easy to maintain, there will be two json reader classes named as JsonReader and JsonLineReader. The former one is the one which directly inherits InputEntityReader and uses ObjectMapper.readValues to parse input text. And JsonLineReader is renamed from the orignal JsonReader which inherits TextReader to process json text line by line. Both of them are instantiated by JsonInputFormat based on the value of lineSplittable.

@jihoonson
Copy link
Copy Markdown
Contributor

@FrankChen021 it sounds good to me 🙂

Comment thread core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java Outdated
@FrankChen021 FrankChen021 changed the title Fix ingestion failure of pretty-formatted JSON message [WIP]Fix ingestion failure of pretty-formatted JSON message Sep 16, 2020
@FrankChen021
Copy link
Copy Markdown
Member Author

FrankChen021 commented Sep 16, 2020

Marked as WIP temporarily.

JsonReader.hasNext, which calls the jackson's MappingIterator.hasNext, throws exception when a json text is ill-formed, this causes some old test cases on sample to fail while test cases on read are successfully passed. This is because stream ingestion handle each record one by one, and if one fails to parse json text, exception is caught and handled gracefully, while sample treats all records as a whole if one record fails, the sample process aborts.

It would take some time to handle this problem.

@jihoonson
Copy link
Copy Markdown
Contributor

@FrankChen021 hmm, do you think it would be better to change the behavior of the sampler? Maybe its behavior should match with the actual data reader.

@FrankChen021
Copy link
Copy Markdown
Member Author

@jihoonson I intended to do that, but after some deep investigation of original implementation, I found that it was not the root cause of previous problem. As a result, exception handling in the new JsonReader.sample was improved, as the way it did in IntermediateRowParsingReader, to eliminate the failure of test cases. So neither test cases nor sample and ingestion behaviors were modified to adapt to this PR.

The PR is now ready for review. I would like hear your voices with regarding to the pass of value of lineSplittable parameter, which I think there should be a more graceful way.

@FrankChen021 FrankChen021 changed the title [WIP]Fix ingestion failure of pretty-formatted JSON message Fix ingestion failure of pretty-formatted JSON message Sep 18, 2020
Copy link
Copy Markdown
Contributor

@jihoonson jihoonson left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankChen021 thanks for the update. I think there is an assumption in the sampler design which should be modified to support the new requirement for parsing multiple JSON objects in an input chunk. I left more detailed comments in line.

Comment thread core/src/main/java/org/apache/druid/data/input/ExceptionThrowingIterator.java Outdated
Comment thread core/src/main/java/org/apache/druid/data/input/ExceptionThrowingIterator.java Outdated
Comment thread core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java Outdated
Comment thread core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java Outdated
Comment thread core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java Outdated
Comment thread core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java Outdated
Comment thread core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java Outdated
Comment thread core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java Outdated
@FrankChen021
Copy link
Copy Markdown
Member Author

@jihoonson The patch has been submitted. The change of signature of IntermediateRowParsingReader#toMap involves changes on all existing subclasses, which are reported insufficient branch or line coverage by CI.

Copy link
Copy Markdown
Contributor

@jihoonson jihoonson left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jihoonson The patch has been submitted. The change of signature of IntermediateRowParsingReader#toMap involves changes on all existing subclasses, which are reported insufficient branch or line coverage by CI.

@FrankChen021 thanks! I left some more comments. For the CI failure, I think we can ignore the test coverage bot for the interface change. However, it seems like there are missing branches in IntermediateRowParsingReader and the new JsonReader as well. Can you please check it?

Diff coverage statistics:
------------------------------------------------------------------------------
|     lines      |    branches    |   functions    |   path
------------------------------------------------------------------------------
| 100% (5/5)     | 100% (2/2)     |  95% (19/20)   | org/apache/druid/data/input/impl/JsonInputFormat.java
|  77% (17/22)   |  25% (1/4)     |  94% (49/52)   | org/apache/druid/data/input/impl/JsonReader.java
|  81% (9/11)    | 100% (0/0)     |  90% (58/64)   | org/apache/druid/data/input/impl/JsonLineReader.java
|   0% (0/1)     | 100% (0/0)     |  50% (1/2)     | org/apache/druid/data/input/impl/DelimitedValueReader.java
|   0% (0/1)     | 100% (0/0)     | 100% (2/2)     | org/apache/druid/data/input/impl/RegexReader.java
|  72% (8/11)    |  33% (2/6)     |  62% (5/8)     | org/apache/druid/data/input/IntermediateRowParsingReader.java
|   0% (0/1)     | 100% (0/0)     |  50% (1/2)     | org/apache/druid/data/input/avro/AvroOCFReader.java
|   0% (0/1)     | 100% (0/0)     |   0% (0/1)     | org/apache/druid/data/input/orc/OrcReader.java
| 100% (1/1)     | 100% (0/0)     | 100% (1/1)     | org/apache/druid/data/input/parquet/ParquetReader.java
------------------------------------------------------------------------------
Total diff coverage:
 - lines: 74% (40/54)
 - branches: 41% (5/12)
 - functions: 89% (136/152)
ERROR: Insufficient branch coverage of 41% (5/12). Required 50%.

Comment thread core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java Outdated
Comment thread core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java Outdated
Comment thread core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java Outdated
Comment thread core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java Outdated
Comment thread core/src/test/java/org/apache/druid/data/input/impl/JsonInputFormatTest.java Outdated
Copy link
Copy Markdown
Contributor

@jihoonson jihoonson left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @FrankChen021, thank you for your patience. I think we are almost there. Please check my last comments.

BTW, now there is only one CI failure:

Diff coverage statistics:
------------------------------------------------------------------------------
|     lines      |    branches    |   functions    |   path
------------------------------------------------------------------------------
|   0% (0/1)     | 100% (0/0)     | 100% (1/1)     | org/apache/druid/metadata/input/SqlReader.java
------------------------------------------------------------------------------
Total diff coverage:
 - lines: 0% (0/1)
 - branches: 100% (0/0)
 - functions: 100% (1/1)
ERROR: Insufficient line coverage of 0% (0/1). Required 50%.

This is because of the interface change you made for sampler, which means the unit test is missing for sampler for SqlReader. I know the missing test is not your fault, but would you mind adding one? It would be pretty straightforward. You can add one in SqlInputSourceTest by mostly replicating testSingleSplit(). All you need to modify are calling sqlReader.sample() instead of read() and verifying InputRowListPlusRawValuess insead of InputRows. But I totally understand even if you don't want to do it in this PR.

Comment on lines 124 to 125
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This doesn't still seem correct to me.. The purpose of returning rawColumn and exception in InputRowListPlusRawValues is showing them in the sampler altogether, so that users will debug their data and input format easily. To do that, we should show exactly what the data was when a ParseException is thrown. Now, thinking about the new behavior parsing a list of JSONs into InputRows, row will contain the list of JSONs here. If a ParseException was thrown while parsing one of them, we should return one InputRowListPlusRawValues which contains row (the whole list of JSONs) and the exception because we don't know exactly in which JSON in the list the ParseException was thrown from. For this, I still think we should change rawValues in InputRowListPlusRawValues to be List<Map<String, Object>>. I understand that you don't want to touch it as it is widely used in unit tests, but I'm not sure if we can fix this without touching it.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree with you.

rawValues in InputRowListPlusRawValues should also be a list to correspond the list of inputRows. Since the rawValues is also passed to SamplerResponseRow when exception occurs, to change rawValues from Map to List<Map> also involves changes of the web console. I don't know if the change of front-end should also be included in this PR.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point. I'm not sure if we have a unit test for parsing sampler response. If we have, the CI will fail and we need to fix it in this PR together. Otherwise, I'm OK with doing the fix in a follow-up. @vogievetsky do you know whether or not we have such a unit test?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should set both rows and rawColumnsList in InputRowListPlusRawValues in this case, so that users will learn how they are different.

@FrankChen021
Copy link
Copy Markdown
Member Author

The branch has been rebased on master because there're some conflicts with it.

@jihoonson Here're some explanation on the latest changes

1st, a new method InputRowListPlusRawValues.ofList is added to create instance of that class by passing list of raw column and rows, while the old of methods are still kept mainly for no changes on old code;

2nd InputRowListPlusRawValues.getRawValues is still kept without involving lots of changes on test cases;

3rd, there's no changes made on existing SamplerResponse interface. All data returned are still row by row. I think keeping this interface contract is reasonable, because its inner container SamplerResponseRow is already a pair of rawColumn and parsed json map.

Copy link
Copy Markdown
Contributor

@jihoonson jihoonson left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankChen021 thanks, sounds good! I reviewed the latest change, but have a concern with potential duplicate rows in the sampler result.

@Nullable
public Map<String, Object> getRawValues()
{
return CollectionUtils.isNullOrEmpty(rawValues) ? null : rawValues.get(0);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: it would be better to use Iterables.getOnlyElement(rawValues) to make sure that you are getting the only element.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But Iterables.getOnlyElement does not check whether the given input is null or not. If the given input is null, there will be a NPE.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I meant, for rawValues.get(0) to make sure there is only one element in there if this method is called.

*/
public static InputRowListPlusRawValues ofList(@Nullable List<Map<String, Object>> rawColumnsList,
@Nullable List<InputRow> inputRows,
ParseException parseException)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ParseException is Nullable.

Comment on lines +148 to +150
responseRows.addAll(rawColumnsList.stream()
.map(rawColumns -> new SamplerResponseRow(rawColumns, null, true, e.getMessage()))
.collect(Collectors.toList()));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can this introduce duplicate rows in responseRows? Suppose you have 3 rawColumns in rawColumnsList including an unparseable row at the second position. The first row will be added to both index and thus a new SamplerResponseRow of the first rawColumns will be added to responseRows. But for the second row, index.add() will throw a ParseException which will execute these lines. In this case, 2 duplicate rawColumns of the first row will be added in responseRows.

Looking at what JsonReader does, it seems throwing away the whole intermediateRow when there is any unparseable row. The sampler behavior should match to the actual ingestion.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I notice that index.add does not throw ParseException but returns the exception in its returning result. The previous version of InputSourceSampler checks the result to rethrow this exception, while the change introduced by #10336 deletes these exception related code, which means the code here won't throw ParseException.

Of course, there's possibility that the code in for-loop throws ParseException in future. I've made little changes to avoid this potential situation to happen. Let's check it once the CI passes.

BTW, my local branch has failed to build since last time it was rebased on master, I have to reply on the CI to check if there're any test case failures.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I notice that index.add does not throw ParseException but returns the exception in its returning result. The previous version of InputSourceSampler checks the result to rethrow this exception, while the change introduced by #10336 deletes these exception related code, which means the code here won't throw ParseException.

Of course, there's possibility that the code in for-loop throws ParseException in future. I've made little changes to avoid this potential situation to happen. Let's check it once the CI passes.

Oops, good point. I forgot what I have done in #10336.. I think index.add() should never throw parseException directly, but return them in IncrementalIndexAddResult. We can add this contract on the Javadoc of IncrementalIndex.add(). I suggest removing the catch clause here to avoid future confusion (there is no point in catching exceptions which cannot be thrown here).

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Got it. BTW, should we handle the exception returned in IncrementalIndexAddResult by adding the corresponding row to responseRows ?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I think so because the sampler result should match to what will actually be ingested to Druid.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A new case testIndexParseException in InputSouceSamplerTest has been added to check whether the exception returned by index.add has been processed properly. And also some little changes are applied in IncrementalIndex.getCombinedParseException and SampleInputRow to make exception message more accurate.

@jihoonson
Copy link
Copy Markdown
Contributor

@FrankChen021 thanks. I left my last comments. Everything else looks good to me.

@lgtm-com
Copy link
Copy Markdown

lgtm-com Bot commented Oct 30, 2020

This pull request introduces 1 alert when merging c1157d2 into 9c51047 - view on LGTM.com

new alerts:

  • 1 for Dereferenced variable may be null

Copy link
Copy Markdown
Contributor

@jihoonson jihoonson left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankChen021, thanks for addressing comments! I have 2 major comments on the test side though. Please take a look.

Comment on lines +126 to +130
if (rawColumnsList != null) {
// add all rows to response
responseRows.addAll(rawColumnsList.stream()
.map(rawColumns -> new SamplerResponseRow(rawColumns, null, true, parseException.getMessage()))
.collect(Collectors.toList()));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: with this change, when a parseException is thrown while parsing a list of rows in one message, those rows in the message will be added as separate rows in responseRows but with the same parseException. This can be confusing since the parseException error message seem irrelevant to the associated rawColumns in the same responseRow. IMO, the better fix in this case would be storing the whole rawColumnsList in one SamplerResponseRow. Then, the parseException can indicate that it was thrown while parsing one of the rows in rawColumnsList. However, this requires a change on the web console side as well. I'm OK with fixing this in a follow-up PR.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, returning the whole rawColumnsList involves some changes at the web-console side which is currently out of my ability. Thanks for understanding.

//well-formed
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8(wellformed)),
//ill-formed
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("illformed")),
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Did you intend StringUtils.toUtf8(illformed)? The variable illformed is not in use.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, illformed here should be a variable instead of a text string.

}

@Test
public void testIndexParseException() throws IOException
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for adding this test! Similar to https://github.com/apache/druid/pull/10383/files#diff-ef25ac1cc1f275b47b939b65e1d0c8b8b8512aeada52d06b8541b8f381df03eeR2731, could you please add a unit test for sampling a block of multiple JSON strings? The unit test can be run only when parserType is STR_JSON. We usually just return in the unit test when the parameter is not what we want to test with. It would be nice if the test verifies the followings:

  • The sampler response when there is no parseException with a list of multiple JSON strings.
  • The sampler response when there is a parseException thrown while parsing a list of multiple JSON strings. Maybe you can improve this unit test to do it as well.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A block of multiple json strings is only supported in stream ingestion as we discussed above, that's also why the test cases are added in kafka-indexing-service module.It's implemented by setting the new property on JsonInputFormat in RecordSupplierInputSource.

For test cases for InputSourceSampler here, the input source is InlineInputSource which always uses InputEntityIteratingReader even when input format is JsonInputFormat, there's no chance to test that. So I think we don't need add test cases here.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we need such tests because KafkaIndexTaskTest only covers the actual indexing side, but not the sampling side. Without those tests, we could break the sampler without being noticed.

For test cases for InputSourceSampler here, the input source is InlineInputSource which always uses InputEntityIteratingReader even when input format is JsonInputFormat, there's no chance to test that. So I think we don't need add test cases here.

Maybe this is not a good place for the new tests, but you don't have to use the InlineInputSource for them. Rather, you should use RecordSupplierInputSource as you mentioned. Since RecordSupplierInputSource accepts the RecordSupplier interface as a parameter which is the data supplier, I think you could implement a mock for testing (I don't think we should test Firehose as it's deprecated). Or do you see some reason you cannot? If it's hard, I'm also OK with adding them in a follow-up.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we need such tests because KafkaIndexTaskTest only covers the actual indexing side, but not the sampling side. Without those tests, we could break the sampler without being noticed.

I forgot that test cases in KafkaIndexTaskTest only cover the indexing process. It's OK for me to add such tests.

BTW, integration tests failed, I don't think it's related to changes in this PR, and the failure can also be seen in my other PR, is there any other problem ?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for understanding. Do you want to add those tests in this PR? For the integration test failure, I don’t think it’s relevant to the changes in this PR. I just retriggered it.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've added a test testMultipleJsonStringInOneBlock, please check it.

Copy link
Copy Markdown
Contributor

@jihoonson jihoonson left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankChen021 thanks for adding a test! All changes good to me, but please fix the CI.

String.join("", STR_JSON_ROWS),

// exclude the last line to form a legal json block
String.join("", STR_JSON_ROWS.stream().limit(STR_JSON_ROWS.size() - 1).collect(Collectors.toList()))
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[ERROR] indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java:1221 -- Can be replaced with 'Collectors.joining'

Seems like the Intellij Inspection CI doesn't like this line.

Copy link
Copy Markdown
Member Author

@FrankChen021 FrankChen021 Nov 11, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Every time I push a commit to a branch that is being merge, I run the test cases in the module which contains the changes in that commit. If it's OK, I'll push the commit. But I find that there's a high probability that CI fails. Sometime it's related to inspection check, sometimes it's caused by failures of test cases in other modules, sometime it's about dependency check, sometime it has something with license check.

I wonder what steps do you follow to check before push a commit ? Do you run all the cases in all modules ? Or is there a simple way to run the checks mentioned above ?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could you re-trigger the CI ? The failure does not related to changes in the PR.

Failed with bad exit code during 'Extraction'

[ERROR] Failed to execute goal on project druid-sql: Could not resolve dependencies for project org.apache.druid:druid-sql:jar:0.21.0-SNAPSHOT: Failed to collect dependencies at org.apache.calcite:calcite-core:jar:1.21.0 -> org.codehaus.janino:janino:jar:3.0.11: Failed to read artifact descriptor for org.codehaus.janino:janino:jar:3.0.11: Could not transfer artifact org.codehaus.janino:janino-parent:pom:3.0.11 from/to central (https://repo.maven.apache.org/maven2): Failed to transfer file https://repo.maven.apache.org/maven2/org/codehaus/janino/janino-parent/3.0.11/janino-parent-3.0.11.pom with status code 503 -> [Help 1]

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Re-triggered LGTM

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @suneet-s , do you have any comments ?

Copy link
Copy Markdown
Contributor

@jihoonson jihoonson Nov 12, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Every time I push a commit to a branch that is being merge, I run the test cases in the module which contains the changes in that commit. If it's OK, I'll push the commit. But I find that there's a high probability that CI fails. Sometime it's related to inspection check, sometimes it's caused by failures of test cases in other modules, sometime it's about dependency check, sometime it has something with license check.

I wonder what steps do you follow to check before push a commit ? Do you run all the cases in all modules ? Or is there a simple way to run the checks mentioned above ?

@FrankChen021 That's what I usually do as well. To be honest, unexpected CI failures make me annoyed too 😅 You can run those checks on your own by running the same command as what Travis runs. You may want to set up some pre-commit/post-commit hooks. The best would be some automatic correction for trivial issues, but I'm not sure if there is such a tool available which is matured and reliable enough.

@jihoonson
Copy link
Copy Markdown
Contributor

I'm merging this PR. Thanks @FrankChen021!

@jihoonson jihoonson merged commit e83d5cb into apache:master Nov 13, 2020
@FrankChen021
Copy link
Copy Markdown
Member Author

I'm merging this PR. Thanks @FrankChen021!

I appreciate your review and comments very much!

abhishekagarwal87 pushed a commit to abhishekagarwal87/druid that referenced this pull request Dec 14, 2020
* scaffolding

* readme

* adjust

* more better, janky heap metadata store, primitive job queue that can submit to overlord, it works - sort of

* test scaffolding

* move InputFormat into IngestSchema

* imply-5135 Create & list ingest tables

* Addressed PR comments

* Removed bean IngestTable

* job processing + sql metadata job table (#78)

* Add indexed-table-loader (#65)

* Add indexed-table-loader

* Fix checkstyle

* Fix intelliJ inspections

* Fix analyze dependencies

* fix license check job

* Add imply-druid-security (#66)

* Add imply-druid-security

* fix checkstyle

* Fix analyze dependencies

* Fix license check job

* Update license header for all imply extensions

* fix intelliJ inspections

* code review

* modify access to protected SQLMetadataConnector methods to allow extensions to create SQL metadata tables using implementation specific constructs (payload type, serial type, etc) (apache#10573)

* Correct getRandomBalancerSegmentHolderTest (apache#10569)

* Add missing docs for timeout exceptions (apache#10554)

* Add missing docs for timeout exceptions

* Add info on auth failures

* Fix ingestion failure of pretty-formatted JSON message (apache#10383)

* support multi-line text

* add test cases

* split json text into lines case by case

* improve exception handle

* fix CI

* use IntermediateRowParsingReader as base of JsonReader

* update doc

* ignore the non-immutable field in test case

* add more test cases

* mark `lineSplittable` as final

* fix testcases

* fix doc

* add a test case for SqlReader

* return all raw columns when exception occurs

* fix CI

* fix test cases

* resolve review comments

* handle ParseException returned by index.add

* apply Iterables.getOnlyElement

* fix CI

* fix test cases

* improve code in more graceful way

* fix test cases

* fix test cases

* add a test case to check multiple json string in one text block

* fix inspection check

* Add TravisCI job that builds and tests on ARM64 CPU architecture (apache#10562)

* Ensure Krb auth before killing YARN apps in graceful shutdown (apache#9785)

* job processing + sql metadata

* Web console: fix data loader schema table column ordering bug and other polish (apache#10588)

* remove unused fields

* keep tables live

* advanced

* fix schema view

* better indication

* tests pass

* Show more instead of show advanced

* fix tests

* extract dynamic configs

* update snapshots

* fix issues

* update snapshot

* reword without >

* some javadoc

* modify druid.historical.cache.maxEntrySize property in Unified format (apache#10590)

Co-authored-by: yuezhang <yuezhang@freewheel.tv>

* Fix license header for imply extensions (#76)

* Fix license header for imply extensions

* arm64 packaging should use jdk8

* maybe this time

* jobs and states and status and whatever

* use indexing client and coordinator client instead of leader client

* always running

* simplify

* fix readme

* Add zero period support to TIMESTAMPADD (apache#10550)

* Allow zero period for TIMESTAMPADD

* update test cases

* add empty zone test case

* add unit test cases for TimestampShiftMacro

* add -Pimply-saas distribution profile, table exists check

* update readme

Co-authored-by: Suneet Saldanha <suneet.saldanha@imply.io>
Co-authored-by: Lucas Capistrant <capistrant@users.noreply.github.com>
Co-authored-by: github-actions <github-actions@github.com>
Co-authored-by: Atul Mohan <atulmohan.mec@gmail.com>
Co-authored-by: frank chen <frank.chen021@outlook.com>
Co-authored-by: Martin Grigorov <martin-g@users.noreply.github.com>
Co-authored-by: Suneet Saldanha <suneet@apache.org>
Co-authored-by: Vadim Ogievetsky <vadim@ogievetsky.com>
Co-authored-by: zhangyue19921010 <69956021+zhangyue19921010@users.noreply.github.com>
Co-authored-by: yuezhang <yuezhang@freewheel.tv>

* fix style and headers

* fix fails

* fix auth

Co-authored-by: Agustin Gonzalez <agustin.gonzalez@imply.io>
Co-authored-by: Suneet Saldanha <suneet.saldanha@imply.io>
Co-authored-by: Lucas Capistrant <capistrant@users.noreply.github.com>
Co-authored-by: github-actions <github-actions@github.com>
Co-authored-by: Atul Mohan <atulmohan.mec@gmail.com>
Co-authored-by: frank chen <frank.chen021@outlook.com>
Co-authored-by: Martin Grigorov <martin-g@users.noreply.github.com>
Co-authored-by: Suneet Saldanha <suneet@apache.org>
Co-authored-by: Vadim Ogievetsky <vadim@ogievetsky.com>
Co-authored-by: zhangyue19921010 <69956021+zhangyue19921010@users.noreply.github.com>
Co-authored-by: yuezhang <yuezhang@freewheel.tv>
@jihoonson jihoonson added this to the 0.21.0 milestone Jan 4, 2021
JulianJaffePinterest pushed a commit to JulianJaffePinterest/druid that referenced this pull request Jan 22, 2021
* support multi-line text

* add test cases

* split json text into lines case by case

* improve exception handle

* fix CI

* use IntermediateRowParsingReader as base of JsonReader

* update doc

* ignore the non-immutable field in test case

* add more test cases

* mark `lineSplittable` as final

* fix testcases

* fix doc

* add a test case for SqlReader

* return all raw columns when exception occurs

* fix CI

* fix test cases

* resolve review comments

* handle ParseException returned by index.add

* apply Iterables.getOnlyElement

* fix CI

* fix test cases

* improve code in more graceful way

* fix test cases

* fix test cases

* add a test case to check multiple json string in one text block

* fix inspection check
@FrankChen021 FrankChen021 deleted the json_parse_bug branch March 11, 2021 10:29
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Kafka ingestion fails to parse multiple-line messages in 0.19

5 participants