Add aggregation function for Largest-Triangle-Three-Buckets (#53145)

* Added a simple lttb aggregate function

* Added support for multiple datatypes

* Added support for Date and Date32, updated LTTBData struct

* Updated code to handle bucket size 0 and 1

* Added sort for LTTBData

* Added tests and documentation

* Added some code style fixes

* Added function to new func ref file

* Removed function from new func ref file

* Apply suggestions from code review

* Updated unit tests

* updated LTTB data code

* Minor style fixes

* Updated code with std sort

* updated tests

* Renamed lttb to largestTriangleThreeBuckets

* Added alias lttb

---------

Co-authored-by: Alexey Milovidov <milovidov@clickhouse.com>
This commit is contained in:
Sinan 2023-10-18 18:17:18 +05:30 committed by GitHub
parent 344c95cfdf
commit c7f78ac73a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 506 additions and 0 deletions

View File

@ -103,4 +103,5 @@ ClickHouse-specific aggregate functions:
- [quantileInterpolatedWeighted](./quantileinterpolatedweighted.md)
- [sparkBar](./sparkbar.md)
- [sumCount](./sumcount.md)
- [largestTriangleThreeBuckets](./largestTriangleThreeBuckets.md)

View File

@ -0,0 +1,67 @@
---
slug: /en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets
sidebar_position: 312
sidebar_label: largestTriangleThreeBuckets
---
# largestTriangleThreeBuckets
Applies the [Largest-Triangle-Three-Buckets](https://skemman.is/bitstream/1946/15343/3/SS_MSthesis.pdf) algorithm to the input data.
The algorithm is used for downsampling time series data for visualization. It is designed to operate on series sorted by x coordinate.
It works by dividing the sorted series into buckets and then finding the largest triangle in each bucket. The number of buckets is equal to the number of points in the resulting series.
the function will sort data by `x` and then apply the downsampling algorithm to the sorted data.
**Syntax**
``` sql
largestTriangleThreeBuckets(n)(x, y)
```
Alias: `lttb`.
**Arguments**
- `x` — x coordinate. [Integer](../../../sql-reference/data-types/int-uint.md) , [Float](../../../sql-reference/data-types/float.md) , [Decimal](../../../sql-reference/data-types/decimal.md) , [Date](../../../sql-reference/data-types/date.md), [Date32](../../../sql-reference/data-types/date32.md), [DateTime](../../../sql-reference/data-types/datetime.md), [DateTime64](../../../sql-reference/data-types/datetime64.md).
- `y` — y coordinate. [Integer](../../../sql-reference/data-types/int-uint.md) , [Float](../../../sql-reference/data-types/float.md) , [Decimal](../../../sql-reference/data-types/decimal.md) , [Date](../../../sql-reference/data-types/date.md), [Date32](../../../sql-reference/data-types/date32.md), [DateTime](../../../sql-reference/data-types/datetime.md), [DateTime64](../../../sql-reference/data-types/datetime64.md).
**Parameters**
- `n` — number of points in the resulting series. [UInt64](../../../sql-reference/data-types/int-uint.md).
**Returned values**
[Array](../../../sql-reference/data-types/array.md) of [Tuple](../../../sql-reference/data-types/tuple.md) with two elements:
**Example**
Input table:
``` text
┌─────x───────┬───────y──────┐
│ 1.000000000 │ 10.000000000 │
│ 2.000000000 │ 20.000000000 │
│ 3.000000000 │ 15.000000000 │
│ 8.000000000 │ 60.000000000 │
│ 9.000000000 │ 55.000000000 │
│ 10.00000000 │ 70.000000000 │
│ 4.000000000 │ 30.000000000 │
│ 5.000000000 │ 40.000000000 │
│ 6.000000000 │ 35.000000000 │
│ 7.000000000 │ 50.000000000 │
└─────────────┴──────────────┘
```
Query:
``` sql
SELECT largestTriangleThreeBuckets(4)(x, y) FROM largestTriangleThreeBuckets_test;
```
Result:
``` text
┌────────largestTriangleThreeBuckets(3)(x, y)───────────┐
│ [(1,10),(3,15),(5,40),(10,70)] │
└───────────────────────────────────────────────────────┘
```

View File

@ -0,0 +1,52 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
namespace DB
{
struct Settings;
namespace
{
AggregateFunctionPtr
createAggregateFunctionLargestTriangleThreeBuckets(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);
if (!(isNumber(argument_types[0]) || isDateOrDate32(argument_types[0]) || isDateTime(argument_types[0])
|| isDateTime64(argument_types[0])))
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Aggregate function {} only supports Date, Date32, DateTime, DateTime64 and Number as the first argument",
name);
if (!(isNumber(argument_types[1]) || isDateOrDate32(argument_types[1]) || isDateTime(argument_types[1])
|| isDateTime64(argument_types[1])))
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Aggregate function {} only supports Date, Date32, DateTime, DateTime64 and Number as the second argument",
name);
return std::make_shared<AggregateFunctionLargestTriangleThreeBuckets>(argument_types, parameters);
}
}
void registerAggregateFunctionLargestTriangleThreeBuckets(AggregateFunctionFactory & factory)
{
factory.registerFunction(AggregateFunctionLargestTriangleThreeBuckets::name, createAggregateFunctionLargestTriangleThreeBuckets);
factory.registerAlias("lttb", AggregateFunctionLargestTriangleThreeBuckets::name);
}
}

View File

@ -0,0 +1,327 @@
#pragma once
#include <iostream>
#include <limits>
#include <numeric>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/StatCommon.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsDateTime.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <base/types.h>
#include <Common/PODArray_fwd.h>
#include <Common/assert_cast.h>
#include <boost/math/distributions/normal.hpp>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
struct LargestTriangleThreeBucketsData : public StatisticalSample<Float64, Float64>
{
void add(const Float64 xval, const Float64 yval, Arena * arena)
{
this->addX(xval, arena);
this->addY(yval, arena);
}
void sort(Arena * arena)
{
// sort the this->x and this->y in ascending order of this->x using index
std::vector<size_t> index(this->x.size());
std::iota(index.begin(), index.end(), 0);
::sort(index.begin(), index.end(), [&](size_t i1, size_t i2) { return this->x[i1] < this->x[i2]; });
SampleX temp_x{};
SampleY temp_y{};
for (size_t i = 0; i < this->x.size(); ++i)
{
temp_x.push_back(this->x[index[i]], arena);
temp_y.push_back(this->y[index[i]], arena);
}
for (size_t i = 0; i < this->x.size(); ++i)
{
this->x[i] = temp_x[i];
this->y[i] = temp_y[i];
}
}
PODArray<std::pair<Float64, Float64>> getResult(size_t total_buckets, Arena * arena)
{
// Sort the data
this->sort(arena);
PODArray<std::pair<Float64, Float64>> result;
// Handle special cases for small data list
if (this->x.size() <= total_buckets)
{
for (size_t i = 0; i < this->x.size(); ++i)
{
result.emplace_back(std::make_pair(this->x[i], this->y[i]));
}
return result;
}
// Handle special cases for 0 or 1 or 2 buckets
if (total_buckets == 0)
return result;
if (total_buckets == 1)
{
result.emplace_back(std::make_pair(this->x.front(), this->y.front()));
return result;
}
if (total_buckets == 2)
{
result.emplace_back(std::make_pair(this->x.front(), this->y.front()));
result.emplace_back(std::make_pair(this->x.back(), this->y.back()));
return result;
}
// Find the size of each bucket
size_t single_bucket_size = this->x.size() / total_buckets;
// Include the first data point
result.emplace_back(std::make_pair(this->x[0], this->y[0]));
for (size_t i = 1; i < total_buckets - 1; ++i) // Skip the first and last bucket
{
size_t start_index = i * single_bucket_size;
size_t end_index = (i + 1) * single_bucket_size;
// Compute the average point in the next bucket
Float64 avg_x = 0;
Float64 avg_y = 0;
for (size_t j = end_index; j < (i + 2) * single_bucket_size; ++j)
{
avg_x += this->x[j];
avg_y += this->y[j];
}
avg_x /= single_bucket_size;
avg_y /= single_bucket_size;
// Find the point in the current bucket that forms the largest triangle
size_t max_index = start_index;
Float64 max_area = 0.0;
for (size_t j = start_index; j < end_index; ++j)
{
Float64 area = std::abs(
0.5
* (result.back().first * this->y[j] + this->x[j] * avg_y + avg_x * result.back().second - result.back().first * avg_y
- this->x[j] * result.back().second - avg_x * this->y[j]));
if (area > max_area)
{
max_area = area;
max_index = j;
}
}
// Include the selected point
result.emplace_back(std::make_pair(this->x[max_index], this->y[max_index]));
}
// Include the last data point
result.emplace_back(std::make_pair(this->x.back(), this->y.back()));
return result;
}
};
class AggregateFunctionLargestTriangleThreeBuckets final : public IAggregateFunctionDataHelper<LargestTriangleThreeBucketsData, AggregateFunctionLargestTriangleThreeBuckets>
{
private:
UInt64 total_buckets{0};
TypeIndex x_type;
TypeIndex y_type;
public:
explicit AggregateFunctionLargestTriangleThreeBuckets(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<LargestTriangleThreeBucketsData, AggregateFunctionLargestTriangleThreeBuckets>({arguments}, {}, createResultType(arguments))
{
if (params.size() != 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require one parameter", getName());
if (params[0].getType() != Field::Types::UInt64)
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a UInt64", getName());
total_buckets = params[0].get<UInt64>();
this->x_type = WhichDataType(arguments[0]).idx;
this->y_type = WhichDataType(arguments[1]).idx;
}
static constexpr auto name = "largestTriangleThreeBuckets";
String getName() const override { return name; }
bool allocatesMemoryInArena() const override { return true; }
static DataTypePtr createResultType(const DataTypes & arguments)
{
TypeIndex x_type = arguments[0]->getTypeId();
TypeIndex y_type = arguments[1]->getTypeId();
UInt32 x_scale = 0;
UInt32 y_scale = 0;
if (const auto * datetime64_type = typeid_cast<const DataTypeDateTime64 *>(arguments[0].get()))
{
x_scale = datetime64_type->getScale();
}
if (const auto * datetime64_type = typeid_cast<const DataTypeDateTime64 *>(arguments[1].get()))
{
y_scale = datetime64_type->getScale();
}
DataTypes types = {getDataTypeFromTypeIndex(x_type, x_scale), getDataTypeFromTypeIndex(y_type, y_scale)};
auto tuple = std::make_shared<DataTypeTuple>(std::move(types));
return std::make_shared<DataTypeArray>(tuple);
}
static DataTypePtr getDataTypeFromTypeIndex(TypeIndex type_index, UInt32 scale)
{
DataTypePtr data_type;
switch (type_index)
{
case TypeIndex::Date:
data_type = std::make_shared<DataTypeDate>();
break;
case TypeIndex::Date32:
data_type = std::make_shared<DataTypeDate32>();
break;
case TypeIndex::DateTime:
data_type = std::make_shared<DataTypeDateTime>();
break;
case TypeIndex::DateTime64:
data_type = std::make_shared<DataTypeDateTime64>(scale);
break;
default:
data_type = std::make_shared<DataTypeNumber<Float64>>();
}
return data_type;
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
Float64 x = getFloat64DataFromColumn(columns[0], row_num, this->x_type);
Float64 y = getFloat64DataFromColumn(columns[1], row_num, this->y_type);
this->data(place).add(x, y, arena);
}
Float64 getFloat64DataFromColumn(const IColumn * column, size_t row_num, TypeIndex type_index) const
{
switch (type_index)
{
case TypeIndex::Date:
return static_cast<const ColumnDate &>(*column).getData()[row_num];
case TypeIndex::Date32:
return static_cast<const ColumnDate32 &>(*column).getData()[row_num];
case TypeIndex::DateTime:
return static_cast<const ColumnDateTime &>(*column).getData()[row_num];
case TypeIndex::DateTime64:
return static_cast<const ColumnDateTime64 &>(*column).getData()[row_num];
default:
return column->getFloat64(row_num);
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & a = this->data(place);
const auto & b = this->data(rhs);
a.merge(b, arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
this->data(place).read(buf, arena);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * arena) const override
{
auto res = this->data(place).getResult(total_buckets, arena);
auto & col = assert_cast<ColumnArray &>(to);
auto & col_offsets = assert_cast<ColumnArray::ColumnOffsets &>(col.getOffsetsColumn());
auto column_x_adder_func = getColumnAdderFunc(x_type);
auto column_y_adder_func = getColumnAdderFunc(y_type);
for (size_t i = 0; i < res.size(); ++i)
{
auto & column_tuple = assert_cast<ColumnTuple &>(col.getData());
column_x_adder_func(column_tuple.getColumn(0), res[i].first);
column_y_adder_func(column_tuple.getColumn(1), res[i].second);
}
col_offsets.getData().push_back(col.getData().size());
}
std::function<void(IColumn &, Float64)> getColumnAdderFunc(TypeIndex type_index) const
{
switch (type_index)
{
case TypeIndex::Date:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnDate &>(column);
col.getData().push_back(static_cast<UInt16>(value));
};
case TypeIndex::Date32:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnDate32 &>(column);
col.getData().push_back(static_cast<UInt32>(value));
};
case TypeIndex::DateTime:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnDateTime &>(column);
col.getData().push_back(static_cast<UInt32>(value));
};
case TypeIndex::DateTime64:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnDateTime64 &>(column);
col.getData().push_back(static_cast<UInt64>(value));
};
default:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnFloat64 &>(column);
col.getData().push_back(value);
};
}
}
};
}

View File

@ -82,6 +82,7 @@ void registerAggregateFunctionIntervalLengthSum(AggregateFunctionFactory &);
void registerAggregateFunctionAnalysisOfVariance(AggregateFunctionFactory &);
void registerAggregateFunctionFlameGraph(AggregateFunctionFactory &);
void registerAggregateFunctionKolmogorovSmirnovTest(AggregateFunctionFactory & factory);
void registerAggregateFunctionLargestTriangleThreeBuckets(AggregateFunctionFactory & factory);
class AggregateFunctionCombinatorFactory;
void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &);
@ -176,6 +177,7 @@ void registerAggregateFunctions()
registerAggregateFunctionAnalysisOfVariance(factory);
registerAggregateFunctionFlameGraph(factory);
registerAggregateFunctionKolmogorovSmirnovTest(factory);
registerAggregateFunctionLargestTriangleThreeBuckets(factory);
registerWindowFunctions(factory);
}

View File

@ -0,0 +1,7 @@
[]
[(1,10)]
[(1,10),(10,70)]
[(1,10),(3,15),(5,40),(10,70)]
[(0.02,0.16),(0.06,0.73),(0.08,0.88),(0.09,0),(0.11,0.31),(0.15,0.09),(0.19,0.27),(0.2,0.24),(0.21,0.46),(0.23,0.23),(0.25,0.34),(0.29,0.16),(0.36,0.99),(0.37,0.86),(0.38,0.75),(0.39,0.86),(0.41,0.63),(0.45,0.86),(0.47,0.06),(0.98,0.09)]
[('2023-01-14 00:00:00.000',35.96528042030847),('2023-03-15 00:00:00.000',98.77709508458238),('2023-06-05 00:00:00.000',8.107958052612418),('2023-09-29 00:00:00.000',80.05338096818797),('2023-12-31 00:00:00.000',98.52375935588333)]
[('2023-01-14 00:00:00.000',35.96528042030847),('2023-03-15 00:00:00.000',98.77709508458238),('2023-06-05 00:00:00.000',8.107958052612418),('2023-09-29 00:00:00.000',80.05338096818797),('2023-12-31 00:00:00.000',98.52375935588333)]

View File

@ -0,0 +1,47 @@
drop table if exists largestTriangleThreeBucketsTestFloat64Float64;
CREATE TABLE largestTriangleThreeBucketsTestFloat64Float64
(
x Float64,
y Float64
) ENGINE = MergeTree order by (y,x);
INSERT INTO largestTriangleThreeBucketsTestFloat64Float64
VALUES (1.0, 10.0),(2.0, 20.0),(3.0, 15.0),(8.0, 60.0),(9.0, 55.0),(10.0, 70.0),(4.0, 30.0),(5.0, 40.0),(6.0, 35.0),(7.0, 50.0);
select largestTriangleThreeBuckets(0)(x, y) FROM largestTriangleThreeBucketsTestFloat64Float64;
select largestTriangleThreeBuckets(1)(x, y) FROM largestTriangleThreeBucketsTestFloat64Float64;
select largestTriangleThreeBuckets(2)(x, y) FROM largestTriangleThreeBucketsTestFloat64Float64;
SELECT largestTriangleThreeBuckets(4)(x, y) AS downsampled_data
FROM largestTriangleThreeBucketsTestFloat64Float64;
drop table largestTriangleThreeBucketsTestFloat64Float64;
drop table if exists largestTriangleThreeBucketsTestDecimal64Decimal64;
CREATE TABLE largestTriangleThreeBucketsTestDecimal64Decimal64
(
x Decimal64(2),
y Decimal64(2)
) ENGINE = MergeTree order by (y,x);
INSERT INTO largestTriangleThreeBucketsTestDecimal64Decimal64(x, y) VALUES (0.63, 0.25), (0.02, 0.16), (0.29, 0.16), (0.2, 0.24), (0.41, 0.63), (0.06, 0.73), (0.36, 0.99), (0.57, 0.18), (0.98, 0.09), (0.73, 0.95), (0.45, 0.86), (0.37, 0.86), (0.6, 0.64), (0.11, 0.31), (0.7, 0.25), (0.85, 0.15), (0.68, 0.39), (0.9, 0.3), (0.25, 0.34), (0.09, 0.0), (0.91, 0.62), (0.47, 0.06), (0.08, 0.88), (0.48, 0.57), (0.55, 0.75), (0.19, 0.27), (0.87, 0.15), (0.15, 0.09), (0.77, 0.28), (0.5, 0.2), (0.39, 0.86), (0.52, 0.11), (0.38, 0.75), (0.71, 0.44), (0.21, 0.46), (0.88, 0.15), (0.83, 0.67), (0.23, 0.23);
select largestTriangleThreeBuckets(20)(x, y) from largestTriangleThreeBucketsTestDecimal64Decimal64;
drop table largestTriangleThreeBucketsTestDecimal64Decimal64;
drop table if exists largestTriangleThreeBucketsTestDateTime64Float64;
create table largestTriangleThreeBucketsTestDateTime64Float64 (x DateTime64(3), y Float64) engine = MergeTree order by (y,x);
INSERT INTO largestTriangleThreeBucketsTestDateTime64Float64 (x, y) VALUES ('2023-09-06 00:00:00', 14.217481939467213), ('2023-09-11 00:00:00', 30.096113766096455), ('2023-01-31 00:00:00', 91.42364224984735), ('2023-12-14 00:00:00', 42.08543753438961), ('2023-10-31 00:00:00', 29.93227107709394), ('2023-12-31 00:00:00', 98.52375935588333), ('2023-07-07 00:00:00', 79.9367415060134), ('2023-08-02 00:00:00', 55.417182033825696), ('2023-03-15 00:00:00', 98.77709508458238), ('2023-09-05 00:00:00', 2.832505232031368), ('2023-06-05 00:00:00', 8.107958052612418), ('2023-02-08 00:00:00', 62.95788480328096), ('2023-02-17 00:00:00', 76.80522155552535), ('2023-11-13 00:00:00', 24.927527306242993), ('2023-02-03 00:00:00', 7.966981342350332), ('2023-05-31 00:00:00', 44.61922229800436), ('2023-09-21 00:00:00', 65.86974701469791), ('2023-01-14 00:00:00', 35.96528042030847), ('2023-02-19 00:00:00', 16.065599678978305), ('2023-05-24 00:00:00', 17.23630978966909), ('2023-11-15 00:00:00', 15.544172190379879), ('2023-12-03 00:00:00', 13.738382187690856), ('2023-10-09 00:00:00', 16.7137129521176), ('2023-11-19 00:00:00', 12.12866001303361), ('2023-06-10 00:00:00', 95.15764263905534), ('2023-07-06 00:00:00', 18.87765798627088), ('2023-03-13 00:00:00', 44.82941460384813), ('2023-01-29 00:00:00', 36.0214717111606), ('2023-12-19 00:00:00', 90.30173319497655), ('2023-07-15 00:00:00', 12.67101467231364), ('2023-07-06 00:00:00', 88.13662733228512), ('2023-05-10 00:00:00', 34.18711141027026), ('2023-11-12 00:00:00', 75.58716684321973), ('2023-10-28 00:00:00', 35.79179186729331), ('2023-11-14 00:00:00', 0.9318182359137728), ('2023-09-29 00:00:00', 80.05338096818797), ('2023-09-13 00:00:00', 16.130217942056866), ('2023-07-28 00:00:00', 11.186638594914744), ('2023-02-12 00:00:00', 69.43690757793445), ('2023-12-18 00:00:00', 12.832032764204616), ('2023-05-21 00:00:00', 74.25002458036471), ('2023-04-03 00:00:00', 51.5662427420719), ('2023-11-27 00:00:00', 96.44359131281784), ('2023-03-29 00:00:00', 33.018594418113324), ('2023-02-07 00:00:00', 84.58945099939815), ('2023-11-16 00:00:00', 40.61531555527268), ('2023-04-21 00:00:00', 60.0545791577218), ('2023-01-31 00:00:00', 87.23185155362057), ('2023-05-19 00:00:00', 77.4095289464808), ('2023-08-26 00:00:00', 18.700816570182067);
select largestTriangleThreeBuckets(5)(x, y) from largestTriangleThreeBucketsTestDateTime64Float64;
select lttb(5)(x, y) from largestTriangleThreeBucketsTestDateTime64Float64;
drop table largestTriangleThreeBucketsTestDateTime64Float64;

View File

@ -1400,6 +1400,7 @@ dmesg
domainWithoutWWW
dont
dotProduct
downsampling
dplyr
dragonbox
dropoff
@ -1708,6 +1709,7 @@ kurtsamp
laion
lang
laravel
largestTriangleThreeBuckets
latencies
ldap
leftPad
@ -1757,6 +1759,7 @@ london
lowcardinality
lowerUTF
lowercased
lttb
lzma
macOS
mailrugo