Skip to content

Inaccurate aggregation results when using doubleSum aggregator when doing a topN query for versions 31.0.0 and 31.0.1 #17813

@climbablebug44

Description

@climbablebug44

Inaccurate aggregation results when using doubleSum aggregator when doing a topN query

Affected Version

31.0.0 and 31.0.1

Description

  • when using rollup metric aggregations on the data(longSum/doubleSum) during ingestion, topN queries with granularities smaller than 15 mins provide inflated results when using doubleSum aggregator alone.

  • when the metric aggregators have a longSum or floatSum along with the doubleSum, it provides correct results.

  • Cluster size

    • mid-size cluster with 3 master services, 6 historicals, 2 brokers
  • Steps to reproduce the problem

    • input data
2025-02-19T00:00:00.000Z,a, east_sc, 1
2025-02-19T00:00:00.000Z,b, east_sc, 1
2025-02-19T00:00:00.000Z,c, east_sc, 1
2025-02-19T00:01:00.000Z,d, east_sc, 2
2025-02-19T00:01:00.000Z,e, east_sc, 2
2025-02-19T00:01:00.000Z,f, east_sc, 2
2025-02-19T00:02:00.000Z,g, east_sc, 3
2025-02-19T00:02:00.000Z,h, east_sc, 3
2025-02-19T00:02:00.000Z,i, east_sc, 3
  • ingestion payload for the data
{
  "type": "index_parallel",
  "id": "index_parallel_topN-bug-check-4_djegegba_2025-02-20T15:54:55.042Z",
  "groupId": "index_parallel_topN-bug-check-4_djegegba_2025-02-20T15:54:55.042Z",
  "resource": {
    "availabilityGroup": "index_parallel_topN-bug-check-4_djegegba_2025-02-20T15:54:55.042Z",
    "requiredCapacity": 1
  },
  "spec": {
    "dataSchema": {
      "dataSource": "topN-bug-check-4",
      "timestampSpec": {
        "column": "time",
        "format": "iso",
        "missingValue": null
      },
      "dimensionsSpec": {
        "dimensions": [],
        "dimensionExclusions": [
          "val",
          "__time",
          "count",
          "time",
          "sum_val"
        ],
        "includeAllDimensions": false,
        "useSchemaDiscovery": false
      },
      "metricsSpec": [
        {
          "type": "count",
          "name": "count"
        },
        {
          "type": "longSum",
          "name": "sum_val",
          "fieldName": "val"
        }
      ],
      "granularitySpec": {
        "type": "uniform",
        "segmentGranularity": "HOUR",
        "queryGranularity": "MINUTE",
        "rollup": true,
        "intervals": []
      },
      "transformSpec": {
        "filter": null,
        "transforms": []
      }
    },
    "ioConfig": {
      "type": "index_parallel",
      "inputSource": {
        "type": "inline",
        "data": "2025-02-19T00:00:00.000Z,a, east_sc, 1\n2025-02-19T00:00:00.000Z,b, east_sc, 1\n2025-02-19T00:00:00.000Z,c, east_sc, 1\n2025-02-19T00:01:00.000Z,d, east_sc, 2\n2025-02-19T00:01:00.000Z,e, east_sc, 2\n2025-02-19T00:01:00.000Z,f, east_sc, 2\n2025-02-19T00:02:00.000Z,g, east_sc, 3\n2025-02-19T00:02:00.000Z,h, east_sc, 3\n2025-02-19T00:02:00.000Z,i, east_sc, 3\n"
      },
      "inputFormat": {
        "type": "csv",
        "columns": [
          "time",
          "tmp",
          "dc",
          "val"
        ]
      },
      "appendToExisting": false,
      "dropExisting": false
    },
    "tuningConfig": {
      "type": "index_parallel",
      "maxRowsPerSegment": 5000000,
      "appendableIndexSpec": {
        "type": "onheap",
        "preserveExistingMetrics": false
      },
      "maxRowsInMemory": 1000000,
      "maxBytesInMemory": 0,
      "skipBytesInMemoryOverheadCheck": false,
      "maxTotalRows": null,
      "numShards": null,
      "splitHintSpec": null,
      "partitionsSpec": {
        "type": "hashed",
        "numShards": null,
        "partitionDimensions": [],
        "partitionFunction": "murmur3_32_abs",
        "maxRowsPerSegment": 5000000
      },
      "indexSpec": {
        "bitmap": {
          "type": "roaring"
        },
        "dimensionCompression": "lz4",
        "stringDictionaryEncoding": {
          "type": "utf8"
        },
        "metricCompression": "lz4",
        "longEncoding": "longs"
      },
      "indexSpecForIntermediatePersists": {
        "bitmap": {
          "type": "roaring"
        },
        "dimensionCompression": "lz4",
        "stringDictionaryEncoding": {
          "type": "utf8"
        },
        "metricCompression": "lz4",
        "longEncoding": "longs"
      },
      "maxPendingPersists": 0,
      "forceGuaranteedRollup": true,
      "reportParseExceptions": false,
      "pushTimeout": 0,
      "segmentWriteOutMediumFactory": null,
      "maxNumConcurrentSubTasks": 1,
      "maxRetry": 3,
      "taskStatusCheckPeriodMs": 1000,
      "chatHandlerTimeout": "PT10S",
      "chatHandlerNumRetries": 5,
      "maxNumSegmentsToMerge": 100,
      "totalNumMergeTasks": 10,
      "logParseExceptions": false,
      "maxParseExceptions": 2147483647,
      "maxSavedParseExceptions": 0,
      "maxColumnsToMerge": -1,
      "awaitSegmentAvailabilityTimeoutMillis": 0,
      "maxAllowedLockCount": -1,
      "numPersistThreads": 1,
      "partitionDimensions": []
    }
  },
  "context": {
    "forceTimeChunkLock": true,
    "useLineageBasedSegmentAllocation": true
  },
  "dataSource": "topN-bug-check-4"
}
  • query payload
{
    "dimension": {
        "type": "default",
        "dimension": "dc",
        "outputName": "Data center"
    },
    "metric": "val",
    "threshold": 5,
    "dataSource": "topN-bug-check-4",
    "granularity": "minute",
    "intervals": [
        "2025-02-19T00:00Z/2025-02-19T00:03Z"
    ],
    "aggregations": [
        {
            "fieldName": "sum_val",
            "type": "doubleSum",
            "name": "val"
        }
    ],
    "postAggregations": [],
    "context": {
        "queryId": "tbc",
        "vectorize": true
    },
    "queryType": "topN"
}
  • wrong aggregation results - there is a mismatch in the original data and the aggregated data when doubleSum is used alone.
    Image

  • this shows the correct aggregation results
    Image

Metadata

Metadata

Assignees

No one assigned

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions