Merge remote-tracking branch 'apache/master' into quidem-runner-extension-submit

This commit is contained in:
Zoltan Haindrich 2024-05-22 18:43:41 +00:00
commit e4ca09d756
17 changed files with 222 additions and 149 deletions

View File

@ -54,6 +54,7 @@ jobs:
build_jdk: 8
runtime_jdk: 8
testing_groups: -Dgroups=${{ matrix.testing_group }}
override_config_path: ./environment-configs/test-groups/prepopulated-data
use_indexer: middleManager
group: ${{ matrix.testing_group }}

View File

@ -63,7 +63,6 @@ services:
service: druid-historical
environment:
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
- AWS_REGION=us-west-2
depends_on:
- druid-zookeeper-kafka

View File

@ -20,7 +20,7 @@
AWS_REGION=us-east-1
# If you are making a change in load list below, make the necessary changes in github actions too
druid_extensions_loadList=["mysql-metadata-storage","druid-s3-extensions","druid-basic-security","simple-client-sslcontext","druid-testing-tools","druid-lookups-cached-global","druid-histogram","druid-datasketches","druid-integration-tests"]
druid_extensions_loadList=["mysql-metadata-storage","druid-s3-extensions","druid-basic-security","simple-client-sslcontext","druid-testing-tools","druid-lookups-cached-global","druid-histogram","druid-datasketches","druid-integration-tests","druid-parquet-extensions","druid-avro-extensions","druid-protobuf-extensions","druid-orc-extensions","druid-kafka-indexing-service"]
# Setting s3 credentials and region to use pre-populated data for testing.
druid_s3_accessKey=AKIAT2GGLKKJQCMG64V4

View File

@ -1371,7 +1371,7 @@
<plugin>
<groupId>org.openrewrite.maven</groupId>
<artifactId>rewrite-maven-plugin</artifactId>
<version>5.27.0</version>
<version>5.31.0</version>
<configuration>
<activeRecipes>
<recipe>org.apache.druid.RewriteRules</recipe>

View File

@ -129,7 +129,6 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig
)
{
this.jsonMapper = jsonMapper;
this.dbTables = dbTables;
this.connector = connector;

View File

@ -23,8 +23,8 @@ const snarkdown = require('snarkdown');
const writefile = 'lib/sql-docs.js';
const MINIMUM_EXPECTED_NUMBER_OF_FUNCTIONS = 167;
const MINIMUM_EXPECTED_NUMBER_OF_DATA_TYPES = 14;
const MINIMUM_EXPECTED_NUMBER_OF_FUNCTIONS = 198;
const MINIMUM_EXPECTED_NUMBER_OF_DATA_TYPES = 15;
const initialFunctionDocs = {
TABLE: [['external', convertMarkdownToHtml('Defines a logical table from an external.')]],
@ -78,6 +78,7 @@ const readDoc = async () => {
await fs.readFile('../docs/querying/sql-array-functions.md', 'utf-8'),
await fs.readFile('../docs/querying/sql-multivalue-string-functions.md', 'utf-8'),
await fs.readFile('../docs/querying/sql-json-functions.md', 'utf-8'),
await fs.readFile('../docs/querying/sql-window-functions.md', 'utf-8'),
await fs.readFile('../docs/querying/sql-operators.md', 'utf-8'),
].join('\n');

View File

@ -62,9 +62,12 @@ function _build_distribution() {
&& cd apache-druid-$(_get_druid_version) \
&& mkdir -p extensions/druid-testing-tools \
&& cp "$(_get_code_root)/extensions-core/testing-tools/target/druid-testing-tools-$(_get_druid_version).jar" extensions/druid-testing-tools/ \
&& echo -e "\n\ndruid.extensions.loadList=[\"druid-hdfs-storage\", \"druid-kafka-indexing-service\", \"druid-datasketches\", \"druid-multi-stage-query\", \"druid-testing-tools\"]" >> conf/druid/single-server/micro-quickstart/_common/common.runtime.properties \
&& mkdir -p extensions/druid-compressed-bigdecimal \
&& cp "$(_get_code_root)/extensions-contrib/compressed-bigdecimal/target/druid-compressed-bigdecimal-$(_get_druid_version).jar" extensions/druid-compressed-bigdecimal/ \
&& echo -e "\n\ndruid.extensions.loadList=[\"druid-hdfs-storage\", \"druid-kafka-indexing-service\", \"druid-multi-stage-query\", \"druid-testing-tools\", \"druid-bloom-filter\", \"druid-datasketches\", \"druid-histogram\", \"druid-stats\", \"druid-compressed-bigdecimal\"]" >> conf/druid/single-server/micro-quickstart/_common/common.runtime.properties \
&& echo -e "\n\ndruid.extensions.loadList=[\"druid-hdfs-storage\", \"druid-kafka-indexing-service\", \"druid-multi-stage-query\", \"druid-testing-tools\", \"druid-bloom-filter\", \"druid-datasketches\", \"druid-histogram\", \"druid-stats\", \"druid-compressed-bigdecimal\"]" >> conf/druid/auto/_common/common.runtime.properties \
&& echo -e "\n\ndruid.server.http.allowedHttpMethods=[\"HEAD\"]" >> conf/druid/single-server/micro-quickstart/_common/common.runtime.properties \
&& echo -e "\n\ndruid.generic.useDefaultValueForNull=false" >> conf/druid/single-server/micro-quickstart/_common/common.runtime.properties \
&& echo -e "\n\ndruid.server.http.allowedHttpMethods=[\"HEAD\"]" >> conf/druid/auto/_common/common.runtime.properties \
)
}

View File

@ -21,6 +21,9 @@ import FileSaver from 'file-saver';
import * as JSONBig from 'json-bigint-native';
import { copyAndAlert, stringifyValue } from './general';
import { queryResultToValuesQuery } from './values-query';
export type Format = 'csv' | 'tsv' | 'json' | 'sql';
export function downloadUrl(url: string, filename: string) {
// Create a link and set the URL using `createObjectURL`
@ -74,44 +77,43 @@ export function downloadFile(text: string, type: string, filename: string): void
FileSaver.saveAs(blob, filename);
}
function queryResultsToString(queryResult: QueryResult, format: string): string {
let lines: string[] = [];
let separator = '';
function queryResultsToString(queryResult: QueryResult, format: Format): string {
const { header, rows } = queryResult;
if (format === 'csv' || format === 'tsv') {
separator = format === 'csv' ? ',' : '\t';
lines.push(
queryResult.header.map(column => formatForFormat(column.name, format)).join(separator),
);
lines = lines.concat(
queryResult.rows.map(r => r.map(cell => formatForFormat(cell, format)).join(separator)),
);
} else {
// json
lines = queryResult.rows.map(r => {
const outputObject: Record<string, any> = {};
for (let k = 0; k < r.length; k++) {
const newName = queryResult.header[k];
if (newName) {
outputObject[newName.name] = r[k];
}
}
return JSONBig.stringify(outputObject);
});
switch (format) {
case 'csv':
case 'tsv': {
const separator = format === 'csv' ? ',' : '\t';
return [
header.map(column => formatForFormat(column.name, format)).join(separator),
...rows.map(r => r.map(cell => formatForFormat(cell, format)).join(separator)),
].join('\n');
}
case 'sql':
return queryResultToValuesQuery(queryResult).toString();
case 'json':
return queryResult
.toObjectArray()
.map(r => JSONBig.stringify(r))
.join('\n');
default:
throw new Error(`unknown format: ${format}`);
}
return lines.join('\n');
}
export function downloadQueryResults(
queryResult: QueryResult,
filename: string,
format: string,
format: Format,
): void {
const resultString: string = queryResultsToString(queryResult, format);
downloadFile(resultString, format, filename);
}
export function copyQueryResultsToClipboard(queryResult: QueryResult, format: string): void {
export function copyQueryResultsToClipboard(queryResult: QueryResult, format: Format): void {
const resultString: string = queryResultsToString(queryResult, format);
copyAndAlert(resultString, 'Query results copied to clipboard');
}

View File

@ -32,8 +32,8 @@ export * from './object-change';
export * from './query-action';
export * from './query-manager';
export * from './query-state';
export * from './sample-query';
export * from './sanitizers';
export * from './sql';
export * from './table-helpers';
export * from './types';
export * from './values-query';

View File

@ -1,82 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import type { Column, LiteralValue, QueryResult, SqlExpression } from '@druid-toolkit/query';
import {
C,
F,
L,
RefName,
SqlAlias,
SqlColumnList,
SqlQuery,
SqlRecord,
SqlValues,
} from '@druid-toolkit/query';
import { oneOf } from './general';
const SAMPLE_ARRAY_SEPARATOR = '<#>'; // Note that this is a regexp so don't add anything that is a special regexp thing
function nullForColumn(column: Column): LiteralValue {
return oneOf(column.sqlType, 'BIGINT', 'DOUBLE', 'FLOAT') ? 0 : '';
}
export function sampleDataToQuery(sample: QueryResult): SqlQuery {
const { header, rows } = sample;
return SqlQuery.create(
new SqlAlias({
expression: SqlValues.create(
rows.map(row =>
SqlRecord.create(
row.map((r, i) => {
if (header[i].nativeType === 'COMPLEX<json>') {
return L(JSON.stringify(r));
} else if (String(header[i].sqlType).endsWith(' ARRAY')) {
return L(r.join(SAMPLE_ARRAY_SEPARATOR));
} else if (r == null || typeof r === 'object') {
// Avoid actually using NULL literals as they create havoc in the VALUES type system and throw errors.
// Also, cleanup array if it happens to get here, it shouldn't.
return L(nullForColumn(header[i]));
} else {
return L(r);
}
}),
),
),
),
alias: RefName.alias('t'),
columns: SqlColumnList.create(header.map((_, i) => RefName.create(`c${i}`, true))),
}),
).changeSelectExpressions(
header.map(({ name, nativeType, sqlType }, i) => {
let ex: SqlExpression = C(`c${i}`);
if (nativeType === 'COMPLEX<json>') {
ex = F('PARSE_JSON', ex);
} else if (sqlType && sqlType.endsWith(' ARRAY')) {
ex = F('STRING_TO_ARRAY', ex, SAMPLE_ARRAY_SEPARATOR);
if (sqlType !== 'VARCHAR ARRAY') {
ex = ex.cast(sqlType);
}
} else if (sqlType) {
ex = ex.cast(sqlType);
}
return ex.as(name, true);
}),
);
}

View File

@ -55,11 +55,11 @@ export function getNumericColumnBraces(
queryResult.header.forEach((column, i) => {
if (!oneOf(column.nativeType, 'LONG', 'FLOAT', 'DOUBLE')) return;
const formatter = columnHints?.get(column.name)?.formatter || formatNumber;
const brace = filterMap(rows, row =>
const braces = filterMap(rows, row =>
oneOf(typeof row[i], 'number', 'bigint') ? formatter(row[i]) : undefined,
);
if (rows.length === brace.length) {
numericColumnBraces[i] = brace;
if (braces.length) {
numericColumnBraces[i] = braces;
}
});
}

View File

@ -41,6 +41,9 @@ export function dataTypeToIcon(dataType: string): IconName {
const typeUpper = dataType.toUpperCase();
switch (typeUpper) {
case 'NULL':
return IconNames.CIRCLE;
case 'TIMESTAMP':
return IconNames.TIME;
@ -75,12 +78,17 @@ export function dataTypeToIcon(dataType: string): IconName {
return IconNames.DIAGRAM_TREE;
case 'COMPLEX<HYPERUNIQUE>':
case 'COMPLEX<HLLSKETCH>':
case 'COMPLEX<HLLSKETCHBUILD>':
case 'COMPLEX<THETASKETCH>':
case 'COMPLEX<THETASKETCHBUILD>':
return IconNames.SNOWFLAKE;
case 'COMPLEX<QUANTILESDOUBLESSKETCH>':
case 'COMPLEX<APPROXIMATEHISTOGRAM>':
case 'COMPLEX<FIXEDBUCKETSHISTOGRAM>':
case 'COMPLEX<ARRAYOFDOUBLESSKETCH>':
case 'COMPLEX<MOMENTSKETCH>':
return IconNames.HORIZONTAL_DISTRIBUTION;
case 'COMPLEX<VARIANCE>':
@ -93,8 +101,15 @@ export function dataTypeToIcon(dataType: string): IconName {
case 'COMPLEX<SERIALIZABLEPAIRLONGSTRING>':
return IconNames.DOUBLE_CHEVRON_RIGHT;
case 'NULL':
return IconNames.CIRCLE;
case 'COMPLEX<BLOOM>':
return IconNames.FILTER_LIST;
case 'COMPLEX<KLLDOUBLESSKETCH>':
case 'COMPLEX<KLLFLOATSSKETCH>':
return IconNames.HURRICANE;
case 'COMPLEX<COMPRESSEDBIGDECIMAL>':
return IconNames.SORT_NUMERICAL_DESC;
default:
if (typeUpper.startsWith('ARRAY')) return IconNames.ARRAY;

View File

@ -18,26 +18,32 @@
import { QueryResult, sane } from '@druid-toolkit/query';
import { sampleDataToQuery } from './sample-query';
import { queryResultToValuesQuery } from './values-query';
describe('sample-query', () => {
describe('queryResultToValuesQuery', () => {
it('works', () => {
const result = QueryResult.fromRawResult(
[
['__time', 'host', 'service', 'msg'],
['LONG', 'STRING', 'STRING', 'COMPLEX<json>'],
['TIMESTAMP', 'VARCHAR', 'VARCHAR', 'OTHER'],
['__time', 'host', 'service', 'msg', 'language', 'nums', 'nulls'],
['LONG', 'STRING', 'STRING', 'COMPLEX<json>', 'ARRAY<STRING>', 'ARRAY<LONG>', 'STRING'],
['TIMESTAMP', 'VARCHAR', 'VARCHAR', 'OTHER', 'ARRAY', 'ARRAY', 'VARCHAR'],
[
'2022-02-01T00:00:00.000Z',
'brokerA.internal',
'broker',
'{"type":"sys","swap/free":1223334,"swap/max":3223334}',
['es', 'es-419'],
[1],
null,
],
[
'2022-02-01T00:00:00.000Z',
'brokerA.internal',
'broker',
'{"type":"query","time":1223,"bytes":2434234}',
['en', 'es', 'es-419'],
[2, 3],
null,
],
],
false,
@ -46,17 +52,20 @@ describe('sample-query', () => {
true,
);
expect(sampleDataToQuery(result).toString()).toEqual(sane`
expect(queryResultToValuesQuery(result).toString()).toEqual(sane`
SELECT
CAST("c0" AS TIMESTAMP) AS "__time",
CAST("c1" AS VARCHAR) AS "host",
CAST("c2" AS VARCHAR) AS "service",
PARSE_JSON("c3") AS "msg"
CAST("c1" AS TIMESTAMP) AS "__time",
CAST("c2" AS VARCHAR) AS "host",
CAST("c3" AS VARCHAR) AS "service",
PARSE_JSON("c4") AS "msg",
STRING_TO_ARRAY("c5", '<#>') AS "language",
CAST(STRING_TO_ARRAY("c6", '<#>') AS BIGINT ARRAY) AS "nums",
CAST(NULL AS VARCHAR) AS "nulls"
FROM (
VALUES
('2022-02-01T00:00:00.000Z', 'brokerA.internal', 'broker', '"{\\"type\\":\\"sys\\",\\"swap/free\\":1223334,\\"swap/max\\":3223334}"'),
('2022-02-01T00:00:00.000Z', 'brokerA.internal', 'broker', '"{\\"type\\":\\"query\\",\\"time\\":1223,\\"bytes\\":2434234}"')
) AS "t" ("c0", "c1", "c2", "c3")
('2022-02-01T00:00:00.000Z', 'brokerA.internal', 'broker', '{"type":"sys","swap/free":1223334,"swap/max":3223334}', 'es<#>es-419', '1', NULL),
('2022-02-01T00:00:00.000Z', 'brokerA.internal', 'broker', '{"type":"query","time":1223,"bytes":2434234}', 'en<#>es<#>es-419', '2<#>3', NULL)
) AS "t" ("c1", "c2", "c3", "c4", "c5", "c6", "c7")
`);
});
});

View File

@ -0,0 +1,121 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import type { Column, QueryResult, SqlExpression } from '@druid-toolkit/query';
import {
C,
F,
L,
RefName,
SqlAlias,
SqlColumnList,
SqlQuery,
SqlRecord,
SqlType,
SqlValues,
} from '@druid-toolkit/query';
import * as JSONBig from 'json-bigint-native';
import { oneOf } from './general';
const SAMPLE_ARRAY_SEPARATOR = '<#>'; // Note that this is a regexp so don't add anything that is a special regexp thing
/**
This function corrects for the legacy behaviour where Druid sometimes returns array columns as
{ sqlType: 'ARRAY', nativeType: 'ARRAY<STRING>' }
instead of the more correct description of
{ sqlType: 'VARCHAR ARRAY', nativeType: 'ARRAY<STRING>' }
use this function to get the effective SQL type of `VARCHAR ARRAY`
*/
function getEffectiveSqlType(column: Column): string | undefined {
const sqlType = column.sqlType;
if (sqlType === 'ARRAY' && String(column.nativeType).startsWith('ARRAY<')) {
return `${SqlType.fromNativeType(String(column.nativeType).slice(6, -1))} ARRAY`;
}
return sqlType;
}
function columnIsAllNulls(rows: readonly unknown[][], columnIndex: number): boolean {
return rows.every(row => row[columnIndex] === null);
}
function isJsonString(x: unknown): boolean {
return typeof x === 'string' && oneOf(x[0], '"', '{', '[');
}
export function queryResultToValuesQuery(sample: QueryResult): SqlQuery {
const { header, rows } = sample;
return SqlQuery.create(
new SqlAlias({
expression: SqlValues.create(
rows.map(row =>
SqlRecord.create(
row.map((r, i) => {
const column = header[i];
const { nativeType } = column;
const sqlType = getEffectiveSqlType(column);
if (nativeType === 'COMPLEX<json>') {
return L(isJsonString(r) ? r : JSONBig.stringify(r));
} else if (String(sqlType).endsWith(' ARRAY')) {
return L(r.join(SAMPLE_ARRAY_SEPARATOR));
} else if (
sqlType === 'OTHER' &&
String(nativeType).startsWith('COMPLEX<') &&
typeof r === 'string' &&
r.startsWith('"') &&
r.endsWith('"')
) {
// r is a JSON encoded base64 string
return L(r.slice(1, -1));
} else if (typeof r === 'object') {
// Cleanup array if it happens to get here, it shouldn't.
return L.NULL;
} else {
return L(r);
}
}),
),
),
),
alias: RefName.alias('t'),
columns: SqlColumnList.create(header.map((_, i) => RefName.create(`c${i + 1}`, true))),
}),
).changeSelectExpressions(
header.map((column, i) => {
const { name, nativeType } = column;
const sqlType = getEffectiveSqlType(column);
// The columnIsAllNulls check is needed due to https://github.com/apache/druid/issues/16456
// Remove it when the issue above is resolved
let ex: SqlExpression = columnIsAllNulls(rows, i) ? L.NULL : C(`c${i + 1}`);
if (nativeType === 'COMPLEX<json>') {
ex = F('PARSE_JSON', ex);
} else if (String(sqlType).endsWith(' ARRAY')) {
ex = F('STRING_TO_ARRAY', ex, SAMPLE_ARRAY_SEPARATOR);
if (sqlType && sqlType !== 'ARRAY' && sqlType !== 'VARCHAR ARRAY') {
ex = ex.cast(sqlType);
}
} else if (sqlType === 'OTHER' && String(nativeType).startsWith('COMPLEX<')) {
ex = F('DECODE_BASE64_COMPLEX', String(nativeType).slice(8, -1), ex);
} else if (sqlType && sqlType !== 'OTHER') {
ex = ex.cast(sqlType);
}
return ex.as(name, true);
}),
);
}

View File

@ -79,7 +79,7 @@ import {
filterMap,
oneOf,
queryDruidSql,
sampleDataToQuery,
queryResultToValuesQuery,
tickIcon,
timeFormatToSql,
wait,
@ -479,7 +479,7 @@ export const SchemaStep = function SchemaStep(props: SchemaStepProps) {
const sampleDataQuery = useMemo(() => {
if (!sampleState.data) return;
return sampleDataToQuery(sampleState.data);
return queryResultToValuesQuery(sampleState.data);
}, [sampleState.data]);
const previewQueryString = useLastDefined(

View File

@ -16,13 +16,14 @@
* limitations under the License.
*/
import { Button, ButtonGroup, Menu, MenuDivider, MenuItem, Position } from '@blueprintjs/core';
import { Button, ButtonGroup, Menu, MenuItem, Position } from '@blueprintjs/core';
import { IconNames } from '@blueprintjs/icons';
import { Popover2 } from '@blueprintjs/popover2';
import type { JSX } from 'react';
import React, { useState } from 'react';
import type { Execution } from '../../../druid-models';
import type { Format } from '../../../utils';
import {
copyQueryResultsToClipboard,
downloadQueryResults,
@ -76,11 +77,11 @@ export const ExecutionSummaryPanel = React.memo(function ExecutionSummaryPanel(
const warningCount = execution?.stages?.getWarningCount();
const handleDownload = (format: string) => {
const handleDownload = (format: Format) => {
downloadQueryResults(queryResult, `results-${execution.id}.${format}`, format);
};
const handleCopy = (format: string) => {
const handleCopy = (format: Format) => {
copyQueryResultsToClipboard(queryResult, format);
};
@ -113,14 +114,18 @@ export const ExecutionSummaryPanel = React.memo(function ExecutionSummaryPanel(
className="download-button"
content={
<Menu>
<MenuDivider title="Download results as..." />
<MenuItem text="CSV" onClick={() => handleDownload('csv')} />
<MenuItem text="TSV" onClick={() => handleDownload('tsv')} />
<MenuItem text="JSON (new line delimited)" onClick={() => handleDownload('json')} />
<MenuDivider title="Copy to clipboard as..." />
<MenuItem text="CSV" onClick={() => handleCopy('csv')} />
<MenuItem text="TSV" onClick={() => handleCopy('tsv')} />
<MenuItem text="JSON (new line delimited)" onClick={() => handleCopy('json')} />
<MenuItem text="Download results as...">
<MenuItem text="CSV" onClick={() => handleDownload('csv')} />
<MenuItem text="TSV" onClick={() => handleDownload('tsv')} />
<MenuItem text="JSON (new line delimited)" onClick={() => handleDownload('json')} />
<MenuItem text="SQL (VALUES)" onClick={() => handleDownload('sql')} />
</MenuItem>
<MenuItem text="Copy to clipboard as...">
<MenuItem text="CSV" onClick={() => handleCopy('csv')} />
<MenuItem text="TSV" onClick={() => handleCopy('tsv')} />
<MenuItem text="JSON (new line delimited)" onClick={() => handleCopy('json')} />
<MenuItem text="SQL (VALUES)" onClick={() => handleCopy('sql')} />
</MenuItem>
</Menu>
}
position={Position.BOTTOM_RIGHT}

View File

@ -620,7 +620,7 @@ export const ResultTablePane = React.memo(function ResultTablePane(props: Result
{numericColumnBraces[i] ? (
<BracedText
className="table-padding"
text={formatNumber(value)}
text={typeof value === 'number' ? formatNumber(value) : String(value)}
braces={numericColumnBraces[i]}
padFractionalPart
/>