Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

stringFirst/stringLast crashes at aggregation time #7243

Closed
glasser opened this issue Mar 12, 2019 · 13 comments · Fixed by #10332
Closed

stringFirst/stringLast crashes at aggregation time #7243

glasser opened this issue Mar 12, 2019 · 13 comments · Fixed by #10332
Labels

Comments

@glasser
Copy link
Contributor

glasser commented Mar 12, 2019

Affected Version

0.13.0-incubating (and maybe older versions)

Description

See discussion on #5789 for background.

Set up a fresh install of Druid 0.13-incubating following the quickstart.

Write this index spec:

{
  "type" : "index",
  "spec" : {
    "dataSchema" : {
      "dataSource" : "wikipedia",
      "parser" : {
        "type" : "string",
        "parseSpec" : {
          "format" : "json",
          "dimensionsSpec" : {},
          "timestampSpec": {
            "column": "time",
            "format": "iso"
          }
        }
      },
    "metricsSpec": [{
      "name": "channel",
      "fieldName": "channel",
      "type": "stringFirst",
      "maxStringBytes": 100
    }],
      "granularitySpec" : {
        "type" : "uniform",
        "segmentGranularity" : "day",
        "queryGranularity" : "hour",
        "intervals" : ["2015-09-12/2015-09-13"],
        "rollup" : true
      }
    },
    "ioConfig" : {
      "type" : "index",
      "firehose" : {
        "type" : "local",
        "baseDir" : "quickstart/tutorial/",
        "filter" : "wikiticker-2015-09-12-sampled.json.gz"
      },
      "appendToExisting" : false
    },
    "tuningConfig" : {
      "type" : "index",
      "targetPartitionSize" : 5000000,
      "maxRowsInMemory" : 1000,
      "forceExtendableShardSpecs" : true
    }
  }
}

and post it:

$ bin/post-index-task --file stringfirst-index.json
Beginning indexing data for wikipedia
Waiting up to 119s for indexing service [http://localhost:8090/] to become available. [Got: <urlopen error [Errno 61] Connection refused> ]
Task started: index_wikipedia_2019-03-12T19:07:26.507Z
Task log:     http://localhost:8090/druid/indexer/v1/task/index_wikipedia_2019-03-12T19:07:26.507Z/log
Task status:  http://localhost:8090/druid/indexer/v1/task/index_wikipedia_2019-03-12T19:07:26.507Z/status
Task index_wikipedia_2019-03-12T19:07:26.507Z still running...
Task index_wikipedia_2019-03-12T19:07:26.507Z still running...
Task index_wikipedia_2019-03-12T19:07:26.507Z still running...
Task index_wikipedia_2019-03-12T19:07:26.507Z still running...
Task finished with status: FAILED

The log reads:

2019-03-12T19:07:45,892 WARN [appenderator_merge_0] org.apache.druid.segment.realtime.appenderator.AppenderatorImpl - Failed to push merged index for segment[wikipedia_2015-09-12T00:00:00.000Z_2015-09-13T00:00:00.000Z_2019-03-12T19:07:26.636Z].
java.lang.ClassCastException: org.apache.druid.query.aggregation.SerializablePairLongString cannot be cast to java.lang.String
	at org.apache.druid.query.aggregation.first.StringFirstAggregateCombiner.reset(StringFirstAggregateCombiner.java:35) ~[druid-processing-0.13.0-incubating.jar:0.13.0-incubating]
	at org.apache.druid.segment.RowCombiningTimeAndDimsIterator.resetCombinedMetrics(RowCombiningTimeAndDimsIterator.java:249) ~[druid-processing-0.13.0-incubating.jar:0.13.0-incubating]
	at org.apache.druid.segment.RowCombiningTimeAndDimsIterator.combineToCurrentTimeAndDims(RowCombiningTimeAndDimsIterator.java:229) ~[druid-processing-0.13.0-incubating.jar:0.13.0-incubating]
	at org.apache.druid.segment.RowCombiningTimeAndDimsIterator.moveToNext(RowCombiningTimeAndDimsIterator.java:191) ~[druid-processing-0.13.0-incubating.jar:0.13.0-incubating]
	at org.apache.druid.segment.IndexMergerV9.mergeIndexesAndWriteColumns(IndexMergerV9.java:492) ~[druid-processing-0.13.0-incubating.jar:0.13.0-incubating]
	at org.apache.druid.segment.IndexMergerV9.makeIndexFiles(IndexMergerV9.java:191) ~[druid-processing-0.13.0-incubating.jar:0.13.0-incubating]
	at org.apache.druid.segment.IndexMergerV9.merge(IndexMergerV9.java:914) ~[druid-processing-0.13.0-incubating.jar:0.13.0-incubating]
	at org.apache.druid.segment.IndexMergerV9.mergeQueryableIndex(IndexMergerV9.java:832) ~[druid-processing-0.13.0-incubating.jar:0.13.0-incubating]
	at org.apache.druid.segment.IndexMergerV9.mergeQueryableIndex(IndexMergerV9.java:810) ~[druid-processing-0.13.0-incubating.jar:0.13.0-incubating]
	at org.apache.druid.segment.realtime.appenderator.AppenderatorImpl.mergeAndPush(AppenderatorImpl.java:719) ~[druid-server-0.13.0-incubating.jar:0.13.0-incubating]
	at org.apache.druid.segment.realtime.appenderator.AppenderatorImpl.lambda$push$1(AppenderatorImpl.java:623) ~[druid-server-0.13.0-incubating.jar:0.13.0-incubating]
	at com.google.common.util.concurrent.Futures$1.apply(Futures.java:713) [guava-16.0.1.jar:?]
	at com.google.common.util.concurrent.Futures$ChainingListenableFuture.run(Futures.java:861) [guava-16.0.1.jar:?]
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_162]
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_162]
	at java.lang.Thread.run(Thread.java:748) [?:1.8.0_162]

@gianm suggested in #5789 that the AggregateCombiner code was just not running at all and that it should always be acting on SerializablePairLongString values rather than Strings. I am not enough of an expert on aggregation to know if that's correct.

Note that I needed to ensure that the task spec specified rollup: true, a large enough queryGranularity so that multiple rows do get combined, and a small enough maxRowsInMemory so that multiple spill files get combined.

@glasser
Copy link
Contributor Author

glasser commented Mar 12, 2019

I'd be happy to fix this, and it's probably a small fix, but I'm lost as to where the appropriate place to add a unit test is. It seems like the issue here is less "a specific class's implementation has a bug" and more "a class implements the wrong design which only fails in a larger context", so it seems like I'd want to write a test that actually uses the larger context that a StringFirstAggregatorFactory is used in, instead of just being a unit test that StringFirstAggregateCombiner implements what we now think it should implement. Anyone have a good idea of where such a test could go? (An integration test seems overkill.)

@glasser
Copy link
Contributor Author

glasser commented Mar 12, 2019

Maybe this is a question that @leventov (who added AggregateCombiners) could help with? I'm wondering where would be a good place to write a test that verifies that a specific AggregateCombiner works in the context of its AggregatoryFactory. My belief is that StringFirstAggregateCombiner (and StringLastAggregateCombiner) doesn't actually act on the right data type, but it doesn't seem like this is something that should be tested just by testing SFAC itself...

@leventov leventov added the Bug label Mar 18, 2019
@leventov
Copy link
Member

The problem is that StringFirstFoldingAggregatorFactory and StringLastFoldingAggregatorFactory have improper makeAggregateCombiner() implementations. They inherit makeAggregateCombiner() from StringFirstAggregatorFactory and StringLastAggregatorFactory, which is wrong.

Related to #6039.

@glasser
Copy link
Contributor Author

glasser commented Mar 25, 2019

I honestly couldn't figure out what the purpose of the StringFirstFoldingAggregatorFactory class was.

For a short term fix (to fix the bug, without the refactors you're suggesting), is the suggestion that StringFirstFoldingAggregatorFactory.makeAggregateCombiner should have return this?

@leventov
Copy link
Member

leventov commented Mar 25, 2019

For a short term fix (to fix the bug, without the refactors you're suggesting), is the suggestion that StringFirstFoldingAggregatorFactory.makeAggregateCombiner should have return this?

It seems to me that no, you cannot return this. Folding aggregators are uniquely different from simple aggregators. You can make simple aggregators kind of accept both (AFAIR some complex aggregators work this way), but it will make the basic aggregation less efficient.

As a short term fix you can override makeAggregateCombiner() in Folding aggregators properly, shouldn't be a very big deal (still without refactoring I'm suggesting).

@glasser
Copy link
Contributor Author

glasser commented Mar 25, 2019

Oh I see. I was getting makeAggregateCombiner() and getCombiningFactory() confused. So make StringFirstFoldingAggregatoryFactory.makeAggregateCombiner() return a new StringFirstFoldingAggregateCombiner which acts on pairs. I still don't know where to write tests that prove that I've written the correct classes, though.

@leventov
Copy link
Member

I cannot think of a better place than StringFirstFoldingAggregatoryFactoryTest, and calling methods like makeAggregateCombiner() manually and applying values manually. Admittedly, these interfaces are hard to test. It needs work. I've opened #7340 about this.

For your patch, it's not obligatory to add such tests, at least because many other aggregator factories currently don't have such tests either.

@glasser
Copy link
Contributor Author

glasser commented Mar 26, 2019

Sure, I guess the problem is I can write tests that validate that the particular classes do what I wrote them to do, but I'm not sure how to write tests that validate that all the different components of aggregating work together properly with the right types, especially as I don't really understand this corner of Druid well. The difference between "does the code do what I think it does" and "does the code implement the right thing".

@DockerLive
Copy link

I saw this message in the latest version :
(Double/Float/Long) First and Last aggregator cannot be used in ingestion spec, and should only be specified as part of queries.

@EricWohlstadter
Copy link

@glasser Hi David, do you know if there is any progress on this issue? We were hoping to use this feature for a use-case but have come across this bug. Otherwise, do you know if there is an undocumented workaround?

@glasser
Copy link
Contributor Author

glasser commented Aug 22, 2019

My attention has been elsewhere but my situation remains the same -- the fix seems trivial but I'd be way more comfortable making it if there was a more integration/end to end test showing that the fix was correct, and I need advice on how to do that.

@CrowdSalat
Copy link

Is there any update on this topic? It would be neat to be able to use this feature at ingestion time. It would allow us to reduce the amount of stored data and hopefully speed up our queries.

@joykent99
Copy link
Contributor

joykent99 commented Aug 28, 2020

We also encountered similar issue. We have attempted the fix proposed in the above conversation. I will try to post a PR for review soon.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging a pull request may close this issue.

6 participants