some changes and fixes on regression methods

This commit is contained in:
Alexander Kozhikhov 2019-05-31 00:59:40 +03:00
parent b9c68cbf3a
commit ef23528452
12 changed files with 78 additions and 56 deletions

View File

@ -7,6 +7,7 @@
#include <Columns/ColumnTuple.h>
#include <Common/FieldVisitors.h>
#include <Common/typeid_cast.h>
#include <Interpreters/convertFieldToType.h>
#include "AggregateFunctionFactory.h"
#include "FactoryHelpers.h"
#include "Helpers.h"
@ -45,11 +46,11 @@ namespace
/// Such default parameters were picked because they did good on some tests,
/// though it still requires to fit parameters to achieve better result
auto learning_rate = Float64(0.01);
auto l2_reg_coef = Float64(0.01);
UInt32 batch_size = 1;
auto learning_rate = Float64(0.00001);
auto l2_reg_coef = Float64(0.1);
UInt32 batch_size = 15;
std::shared_ptr<IWeightsUpdater> weights_updater = std::make_shared<StochasticGradientDescent>();
std::string weights_updater_name = "\'SGD\'";
std::shared_ptr<IGradientComputer> gradient_computer;
if (!parameters.empty())
@ -66,19 +67,8 @@ namespace
}
if (parameters.size() > 3)
{
if (applyVisitor(FieldVisitorToString(), parameters[3]) == "\'SGD\'")
{
weights_updater = std::make_shared<StochasticGradientDescent>();
}
else if (applyVisitor(FieldVisitorToString(), parameters[3]) == "\'Momentum\'")
{
weights_updater = std::make_shared<Momentum>();
}
else if (applyVisitor(FieldVisitorToString(), parameters[3]) == "\'Nesterov\'")
{
weights_updater = std::make_shared<Nesterov>();
}
else
weights_updater_name = applyVisitor(FieldVisitorToString(), parameters[3]);
if (weights_updater_name != "\'SGD\'" && weights_updater_name != "\'Momentum\'" && weights_updater_name != "\'Nesterov\'")
{
throw Exception("Invalid parameter for weights updater", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -100,20 +90,19 @@ namespace
return std::make_shared<Method>(
argument_types.size() - 1,
gradient_computer,
weights_updater,
weights_updater_name,
learning_rate,
l2_reg_coef,
batch_size,
argument_types,
parameters);
}
}
void registerAggregateFunctionMLMethod(AggregateFunctionFactory & factory)
{
factory.registerFunction("linearRegression", createAggregateFunctionMLMethod<FuncLinearRegression>);
factory.registerFunction("logisticRegression", createAggregateFunctionMLMethod<FuncLogisticRegression>);
factory.registerFunction("stochasticLinearRegression", createAggregateFunctionMLMethod<FuncLinearRegression>);
factory.registerFunction("stochasticLogisticRegression", createAggregateFunctionMLMethod<FuncLogisticRegression>);
}
LinearModelData::LinearModelData(
@ -165,11 +154,8 @@ void LinearModelData::returnWeights(IColumn & to) const
= static_cast<ColumnFloat64 &>(arr_to.getData()).getData();
val_to.reserve(old_size + size);
size_t i = 0;
while (i < weights.size())
{
for (size_t i = 0; i != weights.size(); ++i) {
val_to.push_back(weights[i]);
i++;
}
val_to.push_back(bias);
}
@ -217,7 +203,8 @@ void LinearModelData::merge(const DB::LinearModelData & rhs)
void LinearModelData::add(const IColumn ** columns, size_t row_num)
{
/// first column stores target; features start from (columns + 1)
const auto target = (*columns[0])[row_num].get<Float64>();
Float64 target = (*columns[0]).getFloat64(row_num);
/// Here we have columns + 1 as first column corresponds to target value, and others - to features
weights_updater->add_to_batch(
gradient_batch, *gradient_computer, weights, bias, learning_rate, l2_reg_coef, target, columns + 1, row_num);
@ -410,7 +397,7 @@ void LogisticRegression::compute(
Float64 derivative = bias;
for (size_t i = 0; i < weights.size(); ++i)
{
auto value = (*columns[i])[row_num].get<Float64>();
auto value = (*columns[i]).getFloat64(row_num);
derivative += weights[i] * value;
}
derivative *= target;
@ -419,8 +406,8 @@ void LogisticRegression::compute(
batch_gradient[weights.size()] += learning_rate * target / (derivative + 1);
for (size_t i = 0; i < weights.size(); ++i)
{
auto value = (*columns[i])[row_num].get<Float64>();
batch_gradient[i] += learning_rate * target * value / (derivative + 1) - 2 * l2_reg_coef * weights[i];
auto value = (*columns[i]).getFloat64(row_num);
batch_gradient[i] += learning_rate * target * value / (derivative + 1) - 2 * learning_rate * l2_reg_coef * weights[i];
}
}
@ -483,7 +470,7 @@ void LinearRegression::compute(
Float64 derivative = (target - bias);
for (size_t i = 0; i < weights.size(); ++i)
{
auto value = (*columns[i])[row_num].get<Float64>();
auto value = (*columns[i]).getFloat64(row_num);
derivative -= weights[i] * value;
}
derivative *= (2 * learning_rate);
@ -491,8 +478,8 @@ void LinearRegression::compute(
batch_gradient[weights.size()] += derivative;
for (size_t i = 0; i < weights.size(); ++i)
{
auto value = (*columns[i])[row_num].get<Float64>();
batch_gradient[i] += derivative * value - 2 * l2_reg_coef * weights[i];
auto value = (*columns[i]).getFloat64(row_num);
batch_gradient[i] += derivative * value - 2 * learning_rate * l2_reg_coef * weights[i];
}
}

View File

@ -86,7 +86,7 @@ public:
Float64 l2_reg_coef,
Float64 target,
const IColumn ** columns,
size_t row_num) override;
size_t row_num) override ;
void predict(
ColumnVector<Float64>::Container & container,
@ -256,7 +256,7 @@ public:
explicit AggregateFunctionMLMethod(
UInt32 param_num,
std::shared_ptr<IGradientComputer> gradient_computer,
std::shared_ptr<IWeightsUpdater> weights_updater,
std::string weights_updater_name,
Float64 learning_rate,
Float64 l2_reg_coef,
UInt32 batch_size,
@ -268,15 +268,17 @@ public:
, l2_reg_coef(l2_reg_coef)
, batch_size(batch_size)
, gradient_computer(std::move(gradient_computer))
, weights_updater(std::move(weights_updater))
, weights_updater_name(std::move(weights_updater_name))
{
}
/// This function is called when select linearRegression(...) is called
DataTypePtr getReturnType() const override
{
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeFloat64>());
}
/// This function is called from evalMLMethod function for correct predictValues call
DataTypePtr getReturnTypeToPredict() const override
{
return std::make_shared<DataTypeNumber<Float64>>();
@ -284,7 +286,21 @@ public:
void create(AggregateDataPtr place) const override
{
new (place) Data(learning_rate, l2_reg_coef, param_num, batch_size, gradient_computer, weights_updater);
std::shared_ptr<IWeightsUpdater> new_weights_updater;
if (weights_updater_name == "\'SGD\'")
{
new_weights_updater = std::make_shared<StochasticGradientDescent>();
} else if (weights_updater_name == "\'Momentum\'")
{
new_weights_updater = std::make_shared<Momentum>();
} else if (weights_updater_name == "\'Nesterov\'")
{
new_weights_updater = std::make_shared<Nesterov>();
} else
{
throw Exception("Illegal name of weights updater (should have been checked earlier)", ErrorCodes::LOGICAL_ERROR);
}
new (place) Data(learning_rate, l2_reg_coef, param_num, batch_size, gradient_computer, new_weights_updater);
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
@ -307,6 +323,7 @@ public:
+ ". Required: " + std::to_string(param_num + 1),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
/// This cast is correct because column type is based on getReturnTypeToPredict.
auto & column = dynamic_cast<ColumnVector<Float64> &>(to);
this->data(place).predict(column.getData(), block, arguments, context);
@ -328,15 +345,15 @@ private:
Float64 l2_reg_coef;
UInt32 batch_size;
std::shared_ptr<IGradientComputer> gradient_computer;
std::shared_ptr<IWeightsUpdater> weights_updater;
std::string weights_updater_name;
};
struct NameLinearRegression
{
static constexpr auto name = "linearRegression";
static constexpr auto name = "stochasticLinearRegression";
};
struct NameLogisticRegression
{
static constexpr auto name = "logisticRegression";
static constexpr auto name = "stochasticLogisticRegression";
};
}

View File

@ -215,6 +215,12 @@ UInt64 ColumnVector<T>::get64(size_t n) const
return ext::bit_cast<UInt64>(data[n]);
}
template <typename T>
Float64 ColumnVector<T>::getFloat64(size_t n) const
{
return static_cast<Float64>(data[n]);
}
template <typename T>
void ColumnVector<T>::insertRangeFrom(const IColumn & src, size_t start, size_t length)
{

View File

@ -202,6 +202,8 @@ public:
UInt64 get64(size_t n) const override;
Float64 getFloat64(size_t n) const override;
UInt64 getUInt(size_t n) const override
{
return UInt64(data[n]);

View File

@ -91,6 +91,13 @@ public:
throw Exception("Method get64 is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/// If column stores native numeric type, it returns n-th element casted to Float64
/// Is used in regression methods to cast each features into uniform type
virtual Float64 getFloat64(size_t /*n*/) const
{
throw Exception("Method getFloat64 is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/** If column is numeric, return value of n-th element, casted to UInt64.
* For NULL values of Nullable column it is allowed to return arbitrary value.
* Otherwise throw an exception.

View File

@ -17,22 +17,21 @@
<min_time/>
</main_metric>
<create_query>CREATE TABLE test_model engine = Memory as select linearRegressionState(0.0001)(Age, Income, ParamPrice, Robotness, RefererHash) as state from test.hits</create_query>
<create_query>DROP TABLE IF EXISTS test_model</create_query>
<create_query>CREATE TABLE test_model engine = Memory as select stochasticLinearRegressionState(0.0001)(Age, Income, ParamPrice, Robotness, RefererHash) as state from test.hits</create_query>
<!-- Check model fit-->
<query>with (SELECT linearRegressionState(0.0001, 0, 15)(Age, Income, ParamPrice, Robotness, RefererHash) FROM test.hits) as model select toColumnTypeName(model)</query>
<query>WITH (SELECT stochasticLinearRegressionState(0.0001, 0, 15)(Age, Income, ParamPrice, Robotness, RefererHash) FROM test.hits) AS model SELECT 1</query>
<query>SELECT stochasticLinearRegression(Age, Income, ParamPrice, Robotness, RefererHash) FROM test.hits</query>
<!-- Check model fit with Momentum-->
<query>with (SELECT linearRegressionState(0.0001, 0, 15, 'Momentum')(Age, Income, ParamPrice, Robotness, RefererHash) FROM test.hits) as model select toColumnTypeName(model)</query>
<query>WITH (SELECT stochasticLinearRegressionState(0.0001, 0, 15, 'Momentum')(Age, Income, ParamPrice, Robotness, RefererHash) FROM test.hits) AS model SELECT 1</query>
<!-- Check model fit with Nesterov-->
<query>with (SELECT linearRegressionState(0.0001, 0, 15, 'Nesterov')(Age, Income, ParamPrice, Robotness, RefererHash) FROM test.hits) as model select toColumnTypeName(model)</query>
<query>WITH (SELECT stochasticLinearRegressionState(0.0001, 0, 15, 'Nesterov')(Age, Income, ParamPrice, Robotness, RefererHash) FROM test.hits) AS model SELECT 1</query>
<!-- Check model predict-->
<query>with (SELECT state FROM test_model) as model select evalMLMethod(model, Income, ParamPrice, Robotness, RefererHash) from test.hits</query>
<drop_query>DROP TABLE IF EXISTS train_dataset</drop_query>
<drop_query>DROP TABLE IF EXISTS test_dataset</drop_query>
<drop_query>DROP TABLE IF EXISTS test_model</drop_query>
</test>

File diff suppressed because one or more lines are too long

View File

@ -1,2 +1,3 @@
1
1
1

View File

@ -11,7 +11,10 @@ CREATE TABLE IF NOT EXISTS test.defaults
insert into test.defaults values (-3.273, -1.452, 4.267, 20.0, 40.0), (0.121, -0.615, 4.290, 20.0, 40.0);
DROP TABLE IF EXISTS test.model;
create table test.model engine = Memory as select linearRegressionState(0.1, 0.0, 2, 'SGD')(target, param1, param2) as state from test.defaults;
create table test.model engine = Memory as select stochasticLinearRegressionState(0.1, 0.0, 2, 'SGD')(target, param1, param2) as state from test.defaults;
select ans < -61.374 and ans > -61.375 from
(with (select state from remote('127.0.0.1', test.model)) as model select evalMLMethod(model, predict1, predict2) as ans from remote('127.0.0.1', test.defaults));
SELECT 0 < ans[1] and ans[1] < 0.15 and 0.95 < ans[2] and ans[2] < 1.0 and 0 < ans[3] and ans[3] < 0.05 FROM
(SELECT stochasticLinearRegression(0.000001, 0.01, 100)(number, rand() % 100, number) AS ans FROM numbers(1000))

View File

@ -10,7 +10,7 @@ CREATE TABLE IF NOT EXISTS test.defaults
) ENGINE = Memory;
insert into test.defaults values (1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2)
DROP TABLE IF EXISTS test.model;
create table test.model engine = Memory as select logisticRegressionState(0.1, 0.0, 1.0, 'SGD')(target, param1, param2) as state from test.defaults;
create table test.model engine = Memory as select stochasticLogisticRegressionState(0.1, 0.0, 1.0, 'SGD')(target, param1, param2) as state from test.defaults;
select ans < 1.1 and ans > 0.9 from
(with (select state from test.model) as model select evalMLMethod(model, predict1, predict2) as ans from test.defaults limit 2);

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long