Merge pull request #1039 from CartoDB/time-dimensions

Time dimensions
This commit is contained in:
Javier Goizueta 2018-10-09 16:06:06 +02:00 committed by GitHub
commit 09f75441ba
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 899 additions and 62 deletions

View File

@ -1,8 +1,10 @@
# Changelog
## 6.4.1
Released 2018-mm-dd
## 6.5.0
Released 2018-10-dd
New features
- Aggregation time dimensions
## 6.4.0
Released 2018-09-24

View File

@ -134,6 +134,10 @@ of the original dataset applying three different aggregate functions.
> Note that you can use the original column names as names of the result, but all the result column names must be unique. In particular, the names `cartodb_id`, `the_geom`, `the_geom_webmercator` and `_cdb_feature_count` cannot be used for aggregated columns, as they correspond to columns always present in the result.
#### Limitations:
* The iso text format does not admit `starting` or `count` parameters
* Cyclic units (day of the week, etc.) don't admit `count` or `starting` either.
### `resolution`
Defines the cell-size of the spatial aggregation grid. This is equivalent to the [CartoCSS `-torque-resolution`](https://carto.com/docs/carto-engine/cartocss/properties-for-torque/#-torque-resolution-float) property of Torque maps.

View File

@ -1,5 +1,6 @@
const queryUtils = require('../../utils/query-utils');
const AggregationMapConfig = require('../../models/aggregation/aggregation-mapconfig');
const aggregationQuery = require('../../models/aggregation/aggregation-query');
function MapnikLayerStats () {
this._types = {
@ -19,6 +20,9 @@ function columnAggregations(field) {
if (field.type === 'date') { // TODO other types too?
return ['min', 'max'];
}
if (field.type === 'timeDimension') {
return ['min', 'max'];
}
return [];
}
@ -73,7 +77,7 @@ function _geometryType(ctx) {
}
function _columns(ctx) {
if (ctx.metaOptions.columns || ctx.metaOptions.columnStats) {
if (ctx.metaOptions.columns || ctx.metaOptions.columnStats || ctx.metaOptions.dimensions) {
// note: post-aggregation columns are in layer.options.columns when aggregation is present
return queryUtils.queryPromise(ctx.dbConnection, _getSQL(ctx, sql => queryUtils.getQueryLimited(sql, 0)))
.then(res => formatResultFields(ctx.dbConnection, res.fields));
@ -137,51 +141,89 @@ function _sample(ctx, numRows) {
return Promise.resolve();
}
function _columnStats(ctx, columns) {
function _columnsMetadataRequired(options) {
// We need determine the columns of a query
// if either column stats or dimension stats are required,
// since we'll ultimately use the same query to fetch both
return options.columnStats || options.dimensions;
}
function _columnStats(ctx, columns, dimensions) {
if (!columns) {
return Promise.resolve();
}
if (ctx.metaOptions.columnStats) {
if (_columnsMetadataRequired(ctx.metaOptions)) {
let queries = [];
let aggr = [];
queries.push(new Promise(resolve => resolve(columns))); // add columns as first result
Object.keys(columns).forEach(name => {
aggr = aggr.concat(
columnAggregations(columns[name])
.map(fn => `${fn}("${name}") AS "${name}_${fn}"`)
);
if (columns[name].type === 'string') {
const topN = ctx.metaOptions.columnStats.topCategories || 1024;
const includeNulls = ctx.metaOptions.columnStats.hasOwnProperty('includeNulls') ?
ctx.metaOptions.columnStats.includeNulls :
true;
// TODO: ctx.metaOptions.columnStats.maxCategories
// => use PG stats to dismiss columns with more distinct values
queries.push(
queryUtils.queryPromise(
ctx.dbConnection,
_getSQL(ctx, sql => queryUtils.getQueryTopCategories(sql, name, topN, includeNulls))
).then(res => ({ [name]: { categories: res.rows } }))
if (ctx.metaOptions.columnStats) {
queries.push(new Promise(resolve => resolve({ columns }))); // add columns as first result
Object.keys(columns).forEach(name => {
aggr = aggr.concat(
columnAggregations(columns[name])
.map(fn => `${fn}("${name}") AS "${name}_${fn}"`)
);
}
});
if (columns[name].type === 'string') {
const topN = ctx.metaOptions.columnStats.topCategories || 1024;
const includeNulls = ctx.metaOptions.columnStats.hasOwnProperty('includeNulls') ?
ctx.metaOptions.columnStats.includeNulls :
true;
// TODO: ctx.metaOptions.columnStats.maxCategories
// => use PG stats to dismiss columns with more distinct values
queries.push(
queryUtils.queryPromise(
ctx.dbConnection,
_getSQL(ctx, sql => queryUtils.getQueryTopCategories(sql, name, topN, includeNulls))
).then(res => ({ columns: { [name]: { categories: res.rows } } }))
);
}
});
}
const dimensionsStats = {};
let dimensionsInfo = {};
if (ctx.metaOptions.dimensions && dimensions) {
dimensionsInfo = aggregationQuery.infoForOptions({ dimensions });
Object.keys(dimensionsInfo).forEach(dimName => {
const info = dimensionsInfo[dimName];
if (info.type === 'timeDimension') {
dimensionsStats[dimName] = {
params: info.params
};
aggr = aggr.concat(
columnAggregations(info).map(fn => `${fn}(${info.sql}) AS "${dimName}_${fn}"`)
);
}
});
}
queries.push(
queryUtils.queryPromise(
ctx.dbConnection,
_getSQL(ctx, sql => `SELECT ${aggr.join(',')} FROM (${sql}) AS __cdb_query`)
).then(res => {
let stats = {};
let stats = { columns: {}, dimensions: {} };
Object.keys(columns).forEach(name => {
stats[name] = {};
stats.columns[name] = {};
columnAggregations(columns[name]).forEach(fn => {
stats[name][fn] = res.rows[0][`${name}_${fn}`];
stats.columns[name][fn] = res.rows[0][`${name}_${fn}`];
});
});
Object.keys(dimensionsInfo).forEach(name => {
stats.dimensions[name] = stats.dimensions[name] || Object.assign({}, dimensionsStats[name]);
let type = null;
columnAggregations(dimensionsInfo[name]).forEach(fn => {
type = type ||
fieldTypeSafe(ctx.dbConnection, res.fields.find(f => f.name === `${name}_${fn}`));
stats.dimensions[name][fn] = res.rows[0][`${name}_${fn}`];
});
stats.dimensions[name].type = type;
});
return stats;
})
);
return Promise.all(queries).then(results => ({ columns: mergeColumns(results) }));
return Promise.all(queries).then(results => ({
columns: mergeColumns(results.map(r => r.columns)),
dimensions: mergeColumns(results.map( r => r.dimensions))
}));
}
return Promise.resolve({ columns });
}
@ -211,19 +253,17 @@ function fieldType(cname) {
return tname;
}
function fieldTypeSafe(dbConnection, field) {
const cname = dbConnection.typeName(field.dataTypeID);
return cname ? fieldType(cname) : `unknown(${field.dataTypeID})`;
}
// columns are returned as an object { columnName1: { type1: ...}, ..}
// for consistency with SQL API
function formatResultFields(dbConnection, fields = []) {
let nfields = {};
for (let field of fields) {
const cname = dbConnection.typeName(field.dataTypeID);
let tname;
if ( ! cname ) {
tname = 'unknown(' + field.dataTypeID + ')';
} else {
tname = fieldType(cname);
}
nfields[field.name] = { type: tname };
nfields[field.name] = { type: fieldTypeSafe(dbConnection, field) };
}
return nfields;
}
@ -237,7 +277,7 @@ function (layer, dbConnection, callback) {
dbConnection,
preQuery,
aggrQuery,
metaOptions: layer.options.metadata || {}
metaOptions: layer.options.metadata || {},
};
// TODO: could save some queries if queryUtils.getAggregationMetadata() has been used and kept somewhere
@ -248,6 +288,8 @@ function (layer, dbConnection, callback) {
// TODO: add support for sample.exclude option by, in that case, forcing the columns query and
// passing the results to the sample query function.
const dimensions = (layer.options.aggregation || {}).dimensions;
Promise.all([
_estimatedFeatureCount(ctx).then(
({ estimatedFeatureCount }) => _sample(ctx, estimatedFeatureCount)
@ -256,9 +298,10 @@ function (layer, dbConnection, callback) {
_featureCount(ctx),
_aggrFeatureCount(ctx),
_geometryType(ctx),
_columns(ctx).then(columns => _columnStats(ctx, columns))
_columns(ctx).then(columns => _columnStats(ctx, columns, dimensions))
]).then(results => {
callback(null, mergeResults(results));
results = mergeResults(results);
callback(null, results);
}).catch(error => {
callback(error);
});

View File

@ -1,5 +1,8 @@
const timeDimension = require('./time-dimension');
const DEFAULT_PLACEMENT = 'point-sample';
/**
* Returns a template function (function that accepts template parameters and returns a string)
* to generate an aggregation query.
@ -24,6 +27,16 @@ const templateForOptions = (options) => {
return templateFn;
};
function optionsToParams (options) {
return {
sourceQuery: options.query,
res: 256/options.resolution,
columns: options.columns,
dimensions: options.dimensions,
filters: options.filters
};
}
/**
* Generates an aggregation query given the aggregation options:
* - query
@ -38,16 +51,23 @@ const templateForOptions = (options) => {
* When placement, columns or dimensions are specified, columns are aggregated as requested
* (by default only _cdb_feature_count) and with the_geom_webmercator as defined by placement.
*/
const queryForOptions = (options) => templateForOptions(options)({
sourceQuery: options.query,
res: 256/options.resolution,
columns: options.columns,
dimensions: options.dimensions,
filters: options.filters
});
const queryForOptions = (options) => templateForOptions(options)(optionsToParams(options));
module.exports = queryForOptions;
module.exports.infoForOptions = (options) => {
const params = optionsToParams(options);
const dimensions = {};
dimensionNamesAndExpressions(params).forEach(([dimensionName, info]) => {
dimensions[dimensionName] = {
sql: info.sql,
params: info.effectiveParams,
type: info.type
};
});
return dimensions;
};
const SUPPORTED_AGGREGATE_FUNCTIONS = {
'count': {
sql: (column_name, params) => `count(${params.aggregated_column || '*'})`
@ -113,24 +133,56 @@ const aggregateColumnDefs = ctx => {
const aggregateDimensions = ctx => ctx.dimensions || {};
const dimensionNames = (ctx, table) => {
let dimensions = aggregateDimensions(ctx);
if (table) {
return sep(Object.keys(dimensions).map(
dimension_name => `${table}."${dimension_name}"`
));
const timeDimensionParameters = definition => {
// definition.column should correspond to a wrapped date column
const group = definition.group || {};
return {
time: `to_timestamp("${definition.column}")`,
timezone: group.timezone || 'utc',
units: group.units,
count: group.count || 1,
starting: group.starting,
format: definition.format
};
};
// Adapt old-style dimension definitions for backwards compatibility
const adaptDimensionDefinition = definition => {
if (typeof(definition) === 'string') {
return { column: definition };
}
return sep(Object.keys(dimensions).map(dimension_name => {
return `"${dimension_name}"`;
return definition;
};
const dimensionExpression = definition => {
if (definition.group) {
// Currently only time dimensions are supported with parameters
return Object.assign({ type: 'timeDimension' }, timeDimension(timeDimensionParameters(definition)));
} else {
return { sql: `"${definition.column}"` };
}
};
const dimensionNamesAndExpressions = (ctx) => {
let dimensions = aggregateDimensions(ctx);
return Object.keys(dimensions).map(dimensionName => {
const dimension = adaptDimensionDefinition(dimensions[dimensionName]);
const expression = dimensionExpression(dimension);
return [dimensionName, expression];
});
};
const dimensionNames = (ctx, table) => {
return sep(dimensionNamesAndExpressions(ctx).map(([dimensionName]) => {
return table ? `${table}."${dimensionName}"` : `"${dimensionName}"`;
}));
};
const dimensionDefs = ctx => {
let dimensions = aggregateDimensions(ctx);
return sep(Object.keys(dimensions).map(dimension_name => {
const expression = dimensions[dimension_name];
return `"${expression}" AS "${dimension_name}"`;
}));
return sep(
dimensionNamesAndExpressions(ctx)
.map(([dimensionName, expression]) => `${expression.sql} AS "${dimensionName}"`)
);
};
const aggregateFilters = ctx => ctx.filters || {};

View File

@ -0,0 +1,265 @@
// timezones can be defined either by an numeric offset in seconds or by
// a valid (case-insensitive) tz/PG name;
// they include abbreviations defined by PG (which have precedence and
// are fixed offsets, not handling DST) or general names that can handle DST.
function timezone(tz) {
if (isFinite(tz)) {
return `INTERVAL '${tz} seconds'`;
}
return `'${tz}'`;
}
// We assume t is a TIMESTAMP WITH TIME ZONE.
// If this was to be used with a t which is a TIMESTAMP or TIME (no time zone)
// it should be converted with `timezone('utc',t)` to a type with time zone.
// Note that by default CARTO uses timestamp with time zone columns for dates
// and VectorMapConfigAdapter converts them to epoch numbers.
// So, for using this with aggregations, relying on dates & times
// converted to UTC UNIX epoch numbers, apply `to_timestamp` to the
// (converted) column.
function timeExpression(t, tz) {
if (tz !== undefined) {
return `timezone(${timezone(tz)}, ${t})`;
}
return t;
}
function epochWithDefaults(epoch) {
/* jshint maxcomplexity:9 */ // goddammit linter, I like this as is!!
const format = /^(\d\d\d\d)(?:\-?(\d\d)(?:\-?(\d\d)(?:[T\s]?(\d\d)(?:(\d\d)(?:\:(\d\d))?)?)?)?)?$/;
const match = (epoch || '').match(format) || [];
const year = match[1] || '0001';
const month = match[2] || '01';
const day = match[3] || '01';
const hour = match[4] || '00';
const minute = match[5] || '00';
const second = match[6] || '00';
return `${year}-${month}-${day}T${hour}:${minute}:${second}`;
}
// Epoch should be an ISO timestamp literal without time zone
// (it is interpreted as in the defined timzezone for the input time)
// It can be partial, e.g. 'YYYY', 'YYYY-MM', 'YYYY-MM-DDTHH', etc.
// Defaults are applied: YYYY=0001, MM=01, DD=01, HH=00, MM=00, S=00
// It returns a timestamp without time zone
function epochExpression(epoch) {
return `TIMESTAMP '${epoch}'`;
}
const YEARSPAN = "(date_part('year', $t)-date_part('year', $epoch))";
// Note that SECONDSPAN is not a UTC epoch, but an epoch in the specified TZ,
// so we can use it to compute any multiple of seconds with it without using date_part or date_trunc
const SECONDSPAN = "(date_part('epoch', $t) - date_part('epoch', $epoch))";
const serialParts = {
second: {
sql: `FLOOR(${SECONDSPAN})`,
zeroBased: true
},
minute: {
sql: `FLOOR(${SECONDSPAN}/60)`,
zeroBased: true
},
hour: {
sql: `FLOOR(${SECONDSPAN}/3600)`,
zeroBased: true
},
day: {
sql: `1 + FLOOR(${SECONDSPAN}/86400)`,
zeroBased: false
},
week: {
sql: `1 + FLOOR(${SECONDSPAN}/(7*86400))`,
zeroBased: false
},
month: {
sql: `1 + date_part('month', $t) - date_part('month', $epoch) + 12*${YEARSPAN}`,
zeroBased: false
},
quarter: {
sql: `1 + date_part('quarter', $t) - date_part('quarter', $epoch) + 4*${YEARSPAN}`,
zeroBased: false
},
semester: {
sql: `1 + FLOOR((date_part('month', $t) - date_part('month', $epoch))/6) + 2*${YEARSPAN}`,
zeroBased: false
},
trimester: {
sql: `1 + FLOOR((date_part('month', $t) - date_part('month', $epoch))/4) + 3*${YEARSPAN}`,
zeroBased: false
},
year: {
// for the default epoch this coincides with date_part('year', $t)
sql: `1 + ${YEARSPAN}`,
zeroBased: false
},
decade: {
// for the default epoch this coincides with date_part('decade', $t)
sql: `FLOOR((${YEARSPAN} + 1)/10)`,
zeroBased: true
},
century: {
// for the default epoch this coincides with date_part('century', $t)
sql: `1 + FLOOR(${YEARSPAN}/100)`,
zeroBased: false
},
millennium: {
// for the default epoch this coincides with date_part('millennium', $t)
sql: `1 + FLOOR(${YEARSPAN}/1000)`,
zeroBased: false
}
};
function serialSqlExpr(params) {
const { sql, zeroBased } = serialParts[params.units];
const column = timeExpression(params.time, params.timezone);
const epoch = epochExpression(params.starting);
const serial = sql.replace(/\$t/g, column).replace(/\$epoch/g, epoch);
let expr = serial;
if (params.count !== 1) {
if (zeroBased) {
expr = `FLOOR((${expr})/(${params.count}::double precision))::int`;
} else {
expr = `CEIL((${expr})/(${params.count}::double precision))::int`;
}
} else {
expr = `(${expr})::int`;
}
return expr;
}
const isoParts = {
second: `to_char($t, 'YYYY-MM-DD"T"HH:MI:SS')`,
minute: `to_char($t, 'YYYY-MM-DD"T"HH:MI')`,
hour: `to_char($t, 'YYYY-MM-DD"T"HH')`,
day: `to_char($t, 'YYYY-MM-DD')`,
month: `to_char($t, 'YYYY-MM')`,
year: `to_char($t, 'YYYY')`,
week: `to_char($t, 'IYYY-"W"IW')`,
quarter: `to_char($t, 'YYYY-"Q"Q')`,
semester: `to_char($t, 'YYYY"S"') || to_char(CEIL(date_part('month', $t)/6), '9')`,
trimester: `to_char($t, 'YYYY"t"') || to_char(CEIL(date_part('month', $t)/4), '9')`,
decade: `to_char(date_part('decade', $t), '"D"999')`,
century: `to_char($t, '"C"CC')`,
millennium: `to_char(date_part('millennium', $t), '"M"999')`
};
function isoSqlExpr(params) {
const column = timeExpression(params.time, params.timezone);
if (params.count > 1) {
// TODO: it would be sensible to return the ISO of the first unit in the period
throw new Error('Multiple time units not supported for ISO format');
}
return isoParts[params.units].replace(/\$t/g, column);
}
const cyclicParts = {
dayOfWeek: `date_part('isodow', $t)`, // 1 = monday to 7 = sunday;
dayOfMonth: `date_part('day', $t)`, // 1 to 31
dayOfYear: `date_part('doy', $t)`, // 1 to 366
hourOfDay: `date_part('hour', $t)`, // 0 to 23
monthOfYear: `date_part('month', $t)`, // 1 to 12
quarterOfYear: `date_part('quarter', $t)`, // 1 to 4
semesterOfYear: `FLOOR((date_part('month', $t)-1)/6.0) + 1`, // 1 to 2
trimesterOfYear: `FLOOR((date_part('month', $t)-1)/4.0) + 1`, // 1 to 3
weekOfYear: `date_part('week', $t)`, // 1 to 53
minuteOfHour: `date_part('minute', $t)` // 0 to 59
};
function cyclicSqlExpr(params) {
const column = timeExpression(params.time, params.timezone);
return cyclicParts[params.units].replace(/\$t/g, column);
}
const ACCEPTED_PARAMETERS = ['time', 'units', 'timezone', 'count', 'starting', 'format'];
const REQUIRED_PARAMETERS = ['time', 'units'];
function validateParameters(params, checker) {
const errors = [];
const presentParams = Object.keys(params);
const invalidParams = presentParams.filter(param => !ACCEPTED_PARAMETERS.includes(param));
if (invalidParams.length) {
errors.push(`Invalid parameters: ${invalidParams.join(', ')}`);
}
const missingParams = REQUIRED_PARAMETERS.filter(param => !presentParams.includes(param));
if (missingParams.length) {
errors.push(`Missing parameters: ${missingParams.join(', ')}`);
}
const params_errors = checker(params);
errors.push(...params_errors.errors);
if (errors.length) {
throw new Error(`Invalid time dimension:\n${errors.join("\n")}`);
}
return params_errors.params;
}
const VALID_CYCLIC_UNITS = Object.keys(cyclicParts);
const VALID_SERIAL_UNITS = Object.keys(serialParts);
const VALID_ISO_UNITS = Object.keys(isoParts);
function cyclicCheckParams(params) {
const errors = [];
if (!VALID_CYCLIC_UNITS.includes(params.units)) {
errors.push(`Invalid units "${params.units}"`);
}
if (params.count && params.count > 1) {
errors.push(`Count ${params.count} not supported for cyclic ${params.units}`);
}
return { errors: errors, params: params };
}
function serialCheckParams(params) {
const errors = [];
if (!VALID_SERIAL_UNITS.includes(params.units)) {
errors.push(`Invalid grouping units "${params.units}"`);
}
return { errors: errors, params: Object.assign({}, params, { starting: epochWithDefaults(params.starting) }) };
}
function isoCheckParams(params) {
const errors = [];
if (!VALID_ISO_UNITS.includes(params.units)) {
errors.push(`Invalid units "${params.units}"`);
}
if (params.starting) {
errors.push("Parameter 'starting' not supported for ISO format");
}
return { errors: errors, params: params };
}
const CLASSIFIERS = {
cyclic: {
sqlExpr: cyclicSqlExpr,
checkParams: cyclicCheckParams
},
iso: {
sqlExpr: isoSqlExpr,
checkParams: isoCheckParams
},
serial: {
sqlExpr: serialSqlExpr,
checkParams: serialCheckParams
}
};
function isCyclic(units) {
return VALID_CYCLIC_UNITS.includes(units);
}
function classifierFor(params) {
let classifier = 'serial';
if (params.units && isCyclic(params.units)) {
classifier = 'cyclic';
} else if (params.format === 'iso') {
classifier = 'iso';
}
return CLASSIFIERS[classifier];
}
function classificationSql(params) {
const classifier = classifierFor(params);
params = validateParameters(params, classifier.checkParams);
return { sql: classifier.sqlExpr(params), effectiveParams: params };
}
module.exports = classificationSql;

View File

@ -95,7 +95,14 @@ module.exports = class AggregationMapConfigAdapter {
const sqlQueryWrap = layer.options.sql_wrap;
let aggregationSql = mapConfig.getAggregatedQuery(index);
let aggregationSql;
try {
aggregationSql = mapConfig.getAggregatedQuery(index);
}
catch (error) {
return reject(error);
}
if (sqlQueryWrap) {
aggregationSql = sqlQueryWrap.replace(/<%=\s*sql\s*%>/g, aggregationSql);

View File

@ -16,6 +16,36 @@ if (process.env.POSTGIS_VERSION >= '20400') {
});
}
// Generate points with values and times.
// The point location is spanned over a given length, by default it is 0 so
// all points have the same location, which can be used to test aggregation dimensions
// the default point is in tile
function pointsWithTimeSQL(n, startTime, endTime, span = 0, x0 = 0.1, y0 = 0.1) {
return `
WITH params AS (
SELECT
'${startTime}'::timestamp with time zone AS min_t,
'${endTime}'::timestamp with time zone AS max_t,
${x0} AS x0, ${y0} AS y0,
${span} AS length,
${n} AS n
),
positions AS (
SELECT
step::float8/n AS s,
x0 + (step::float8/n - 0.5)*length AS x, y0 AS y
FROM params, generate_series(1, n) AS step
)
SELECT
row_number() over () AS cartodb_id,
n*10 AS value,
min_t + (max_t - min_t)*s AS date,
ST_SetSRID(ST_MakePoint(x, y), 4326) AS the_geom,
ST_Transform(ST_SetSRID(ST_MakePoint(x, y), 4326), 3857) AS the_geom_webmercator
FROM params, positions
`;
}
describe('aggregation', function () {
const POINTS_SQL_1 = `
@ -878,6 +908,440 @@ describe('aggregation', function () {
});
});
it('time dimensions', function (done) {
this.mapConfig = createVectorMapConfig([
{
type: 'cartodb',
options: {
sql: POINTS_SQL_TIMESTAMP_1,
dates_as_numbers: true,
aggregation: {
threshold: 1,
dimensions: {
dow: {
column: 'date',
group: {
units: 'dayOfWeek'
}
}
}
}
}
}
]);
this.testClient = new TestClient(this.mapConfig);
const options = {
format: 'mvt'
};
this.testClient.getTile(0, 0, 0, options, (err, res, tile) => {
if (err) {
return done(err);
}
const tileJSON = tile.toJSON();
tileJSON[0].features.forEach(feature => assert.equal(typeof feature.properties.dow, 'number'));
done();
});
});
it('aggregation dimensions only used if present', function (done) {
const nPoints = 50;
this.mapConfig = createVectorMapConfig([
{
type: 'cartodb',
options: {
sql: pointsWithTimeSQL(nPoints, '2000-01-01T00:00:00+00', '2019-12-31T23:59:59+00', 0),
dates_as_numbers: true,
aggregation: {
threshold: 1,
}
}
}
]);
this.testClient = new TestClient(this.mapConfig);
const options = {
format: 'mvt'
};
this.testClient.getTile(0, 0, 0, options, (err, res, tile) => {
if (err) {
return done(err);
}
const tileJSON = tile.toJSON();
// Everything's aggregated into a single feature because the only
// dimension is space and all points are in the same place.
assert.deepEqual(tileJSON[0].features.map(f => f.properties._cdb_feature_count), [nPoints]);
done();
});
});
it('aggregation dimension year used', function (done) {
const nPoints = 50;
this.mapConfig = createVectorMapConfig([
{
type: 'cartodb',
options: {
sql: pointsWithTimeSQL(nPoints, '2000-01-01T00:00:00+00', '2019-12-31T23:59:59+00', 0),
dates_as_numbers: true,
aggregation: {
threshold: 1,
dimensions: {
year: {
column: 'date',
group: {
units: 'year'
}
}
}
},
}
}
]);
this.testClient = new TestClient(this.mapConfig);
const options = {
format: 'mvt'
};
this.testClient.getTile(0, 0, 0, options, (err, res, tile) => {
if (err) {
return done(err);
}
const tileJSON = tile.toJSON();
// Now all features have same location, but the year is an additional dimension
// with 20 different values, so we'll have an aggregated feature for each.
const expectedYears = Array.from({length: 20}, (_, k) => 2000 + k); // 2000 to 2019
const resultYears = tileJSON[0].features.map(f => f.properties.year).sort((a, b) => a - b);
assert.deepEqual(resultYears, expectedYears);
done();
});
});
it('aggregation dimension month with count', function (done) {
this.mapConfig = createVectorMapConfig([
{
type: 'cartodb',
options: {
sql: pointsWithTimeSQL(50, '2018-01-01T00:00:00+00', '2018-12-31T23:59:59+00', 0),
dates_as_numbers: true,
aggregation: {
threshold: 1,
dimensions: {
month: {
column: 'date',
group: {
units: 'month',
count: 5,
starting: '2018-01'
}
}
}
},
}
}
]);
this.testClient = new TestClient(this.mapConfig);
const options = {
format: 'mvt'
};
this.testClient.getTile(0, 0, 0, options, (err, res, tile) => {
if (err) {
return done(err);
}
const tileJSON = tile.toJSON();
assert.equal(tileJSON[0].features.length, 3);
const resultQuimesters = tileJSON[0].features.map(f => f.properties.month).sort((a, b) => a - b);
assert.deepEqual(resultQuimesters, [1, 2, 3]);
done();
});
});
it('aggregation dimension month with starting', function (done) {
this.mapConfig = createVectorMapConfig([
{
type: 'cartodb',
options: {
sql: pointsWithTimeSQL(50, '2018-01-01T00:00:00+00', '2018-12-31T23:59:59+00', 0),
dates_as_numbers: true,
aggregation: {
threshold: 1,
dimensions: {
month: {
column: 'date',
group: {
units: 'month',
starting: '2017-01'
}
}
}
}
}
}
]);
this.testClient = new TestClient(this.mapConfig);
const options = {
format: 'mvt'
};
this.testClient.getTile(0, 0, 0, options, (err, res, tile) => {
if (err) {
return done(err);
}
const tileJSON = tile.toJSON();
const resultMonths = tileJSON[0].features.map(f => f.properties.month).sort((a, b) => a - b);
assert.deepEqual(resultMonths, [13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24]);
done();
});
});
it('aggregation dimension month by default UTC', function (done) {
this.mapConfig = createVectorMapConfig([
{
type: 'cartodb',
options: {
sql: pointsWithTimeSQL(50, '2018-01-01T00:00:00+00', '2018-01-31T23:59:59+00', 0),
dates_as_numbers: true,
aggregation: {
threshold: 1,
dimensions: {
dow: {
column: 'date',
group: {
units: 'month',
timezone: '+00'
}
}
}
}
}
}
]);
this.testClient = new TestClient(this.mapConfig);
const options = {
format: 'mvt'
};
this.testClient.getTile(0, 0, 0, options, (err, res, tile) => {
if (err) {
return done(err);
}
const tileJSON = tile.toJSON();
// In UTC all times are in the same month 2018-01
assert.equal(tileJSON[0].features.length, 1);
done();
});
});
it('aggregation dimension month with timezone', function (done) {
this.mapConfig = createVectorMapConfig([
{
type: 'cartodb',
options: {
sql: pointsWithTimeSQL(50, '2018-01-01T00:00:00+00', '2018-01-31T23:59:59+00', 0),
dates_as_numbers: true,
aggregation: {
threshold: 1,
dimensions: {
dow: {
column: 'date',
group: {
units: 'month',
timezone: '+7200'
}
}
}
}
}
}
]);
this.testClient = new TestClient(this.mapConfig);
const options = {
format: 'mvt'
};
this.testClient.getTile(0, 0, 0, options, (err, res, tile) => {
if (err) {
return done(err);
}
const tileJSON = tile.toJSON();
// In UTC+2 some times are in a different month
assert.equal(tileJSON[0].features.length, 2);
done();
});
});
it('time dimensions stats', function (done) {
this.mapConfig = createVectorMapConfig([
{
type: 'cartodb',
options: {
sql: POINTS_SQL_TIMESTAMP_1,
dates_as_numbers: true,
aggregation: {
threshold: 1,
dimensions: {
dow: {
column: 'date',
group: {
units: 'dayOfWeek'
}
}
}
},
metadata: {
dimensions: true
}
}
}
]);
this.testClient = new TestClient(this.mapConfig);
this.testClient.getLayergroup(function(err, layergroup) {
assert.ifError(err);
const expectedDimensions = {
dow:
{ params:
{ time: 'to_timestamp("date")',
timezone: 'utc',
units: 'dayOfWeek',
count: 1 },
min: 4,
max: 7,
type: 'number' }
};
assert.deepEqual(layergroup.metadata.layers[0].meta.stats.dimensions, expectedDimensions);
done();
});
});
it('no time dimensions stats by default', function (done) {
this.mapConfig = createVectorMapConfig([
{
type: 'cartodb',
options: {
sql: POINTS_SQL_TIMESTAMP_1,
dates_as_numbers: true,
aggregation: {
threshold: 1,
dimensions: {
dow: {
column: 'date',
group: {
units: 'dayOfWeek'
}
}
}
}
}
}
]);
this.testClient = new TestClient(this.mapConfig);
this.testClient.getLayergroup(function(err, layergroup) {
assert.ifError(err);
assert(!layergroup.metadata.layers[0].meta.stats.dimensions);
done();
});
});
it('aggregation dimension month iso format', function (done) {
this.mapConfig = createVectorMapConfig([
{
type: 'cartodb',
options: {
sql: pointsWithTimeSQL(50, '2018-01-01T00:00:00+00', '2018-12-31T23:59:59+00', 0),
dates_as_numbers: true,
aggregation: {
threshold: 1,
dimensions: {
month: {
column: 'date',
group: {
units: 'month',
},
format: 'iso'
}
}
}
}
}
]);
this.testClient = new TestClient(this.mapConfig);
const options = {
format: 'mvt'
};
this.testClient.getTile(0, 0, 0, options, (err, res, tile) => {
if (err) {
return done(err);
}
const tileJSON = tile.toJSON();
const resultMonths = tileJSON[0].features.map(f => f.properties.month).sort();
assert.deepEqual(resultMonths, [
'2018-01', '2018-02', '2018-03', '2018-04', '2018-05','2018-06',
'2018-07', '2018-08', '2018-09', '2018-10', '2018-11', '2018-12'
]);
done();
});
});
it('aggregation dimension month iso format with timezone', function (done) {
this.mapConfig = createVectorMapConfig([
{
type: 'cartodb',
options: {
sql: pointsWithTimeSQL(50, '2018-01-01T00:00:00+00', '2018-12-31T23:59:59+00', 0),
dates_as_numbers: true,
aggregation: {
threshold: 1,
dimensions: {
month: {
column: 'date',
group: {
units: 'month',
timezone: '+7200'
},
format: 'iso',
}
}
}
}
}
]);
this.testClient = new TestClient(this.mapConfig);
const options = {
format: 'mvt'
};
this.testClient.getTile(0, 0, 0, options, (err, res, tile) => {
if (err) {
return done(err);
}
const tileJSON = tile.toJSON();
const resultMonths = tileJSON[0].features.map(f => f.properties.month).sort();
assert.deepEqual(resultMonths, [
'2018-01', '2018-02', '2018-03', '2018-04', '2018-05', '2018-06',
'2018-07', '2018-08', '2018-09', '2018-10', '2018-11', '2018-12',
'2019-01'
]);
done();
});
});
['centroid', 'point-sample', 'point-grid'].forEach(placement => {
it(`dimensions should work for ${placement} placement`, function(done) {
this.mapConfig = createVectorMapConfig([