Merge branch 'master' of https://github.com/yandex/ClickHouse into CLICKHOUSE-2720

This commit is contained in:
Ivan Blinkov 2018-07-18 16:24:31 +03:00
commit 1c17dcaa21
31 changed files with 255 additions and 101 deletions

View File

@ -72,6 +72,7 @@
### Backward incompatible changes:
* Removed escaping in `Vertical` and `Pretty*` formats and deleted the `VerticalRaw` format.
* If servers with version 1.1.54388 (or newer) and servers with older version are used simultaneously in distributed query and the query has `cast(x, 'Type')` expression in the form without `AS` keyword and with `cast` not in uppercase, then the exception with message like `Not found column cast(0, 'UInt8') in block` will be thrown. Solution: update server on all cluster nodes.
## ClickHouse release 1.1.54385, 2018-06-01

View File

@ -66,6 +66,7 @@
### Обратно несовместимые изменения:
* Убран escaping в форматах `Vertical` и `Pretty*`, удалён формат `VerticalRaw`.
* Если в распределённых запросах одновременно участвуют серверы версии 1.1.54388 или новее и более старые, то при использовании выражения `cast(x, 'Type')`, записанного без указания `AS`, если слово `cast` указано не в верхнем регистре, возникает ошибка вида `Not found column cast(0, 'UInt8') in block`. Решение: обновить сервер на всём кластере.
## ClickHouse release 1.1.54385, 2018-06-01

View File

@ -1,5 +1,13 @@
# Freebsd: /usr/local/include/libcpuid/libcpuid_types.h:61:29: error: conflicting declaration 'typedef long long int int64_t'
# TODO: test new libcpuid - maybe already fixed
if (NOT ARCH_ARM)
option (USE_INTERNAL_CPUID_LIBRARY "Set to FALSE to use system cpuid library instead of bundled" ${NOT_UNBUNDLED})
if (ARCH_FREEBSD)
set (DEFAULT_USE_INTERNAL_CPUID_LIBRARY 1)
else ()
set (DEFAULT_USE_INTERNAL_CPUID_LIBRARY ${NOT_UNBUNDLED})
endif ()
option (USE_INTERNAL_CPUID_LIBRARY "Set to FALSE to use system cpuid library instead of bundled" ${DEFAULT_USE_INTERNAL_CPUID_LIBRARY})
endif ()
#if (USE_INTERNAL_CPUID_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libcpuid/include/cpuid/libcpuid.h")

View File

@ -1,24 +1,25 @@
# This strings autochanged from release_lib.sh:
set(VERSION_DESCRIBE v1.1.54394-testing)
set(VERSION_REVISION 54394)
set(VERSION_GITHASH 875ea0f4eaa3592f1fe628b6a1150d91b04ad574)
set(VERSION_REVISION 54395 CACHE STRING "")
set(VERSION_MAJOR 1 CACHE STRING "")
set(VERSION_MINOR 1 CACHE STRING "")
set(VERSION_PATCH 54398 CACHE STRING "")
set(VERSION_GITHASH 4b31f389b743c69af688788c0d0cdb8973aefa77 CACHE STRING "")
set(VERSION_DESCRIBE v1.1.54398-testing CACHE STRING "")
set(VERSION_STRING 1.1.54398 CACHE STRING "")
# end of autochange
set (VERSION_MAJOR 1)
set (VERSION_MINOR 1)
set (VERSION_PATCH ${VERSION_REVISION})
set (VERSION_EXTRA "")
set (VERSION_TWEAK "")
set(VERSION_EXTRA "" CACHE STRING "")
set(VERSION_TWEAK "" CACHE STRING "")
set (VERSION_STRING "${VERSION_MAJOR}.${VERSION_MINOR}.${VERSION_PATCH}")
if (VERSION_TWEAK)
set(VERSION_STRING "${VERSION_STRING}.${VERSION_TWEAK}")
string(CONCAT VERSION_STRING ${VERSION_STRING} "." ${VERSION_TWEAK})
endif ()
if (VERSION_EXTRA)
set(VERSION_STRING "${VERSION_STRING}${VERSION_EXTRA}")
string(CONCAT VERSION_STRING ${VERSION_STRING} "." ${VERSION_EXTRA})
endif ()
set (VERSION_FULL "${PROJECT_NAME} ${VERSION_STRING}")
set (VERSION_NAME "${PROJECT_NAME}")
set (VERSION_FULL "${VERSION_NAME} ${VERSION_STRING}")
if (APPLE)
# dirty hack: ld: malformed 64-bit a.b.c.d.e version number: 1.1.54160

View File

@ -28,6 +28,7 @@
#include <Common/StringUtils/StringUtils.h>
#include <Common/typeid_cast.h>
#include <Common/Config/ConfigProcessor.h>
#include <Common/config_version.h>
#include <Core/Types.h>
#include <Core/QueryProcessingStage.h>
#include <IO/ReadBufferFromFileDescriptor.h>
@ -1316,10 +1317,7 @@ private:
void showClientVersion()
{
std::cout << "ClickHouse client version " << DBMS_VERSION_MAJOR
<< "." << DBMS_VERSION_MINOR
<< "." << ClickHouseRevision::get()
<< "." << std::endl;
std::cout << DBMS_NAME << " client version " << VERSION_STRING << "." << std::endl;
}
public:

View File

@ -17,6 +17,7 @@
#include <Common/Config/ConfigProcessor.h>
#include <Common/escapeForFileName.h>
#include <Common/ClickHouseRevision.h>
#include <Common/config_version.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteBufferFromFileDescriptor.h>
@ -355,10 +356,7 @@ void LocalServer::setupUsers()
static void showClientVersion()
{
std::cout << "ClickHouse client version " << DBMS_VERSION_MAJOR
<< "." << DBMS_VERSION_MINOR
<< "." << ClickHouseRevision::get()
<< "." << std::endl;
std::cout << DBMS_NAME << " client version " << VERSION_STRING << "." << std::endl;
}
std::string LocalServer::getHelpHeader() const

View File

@ -1,36 +1,27 @@
#include "TCPHandler.h"
#include <iomanip>
#include <Poco/Net/NetException.h>
#include <Common/ClickHouseRevision.h>
#include <Common/Stopwatch.h>
#include <IO/Progress.h>
#include <IO/CompressedReadBuffer.h>
#include <IO/CompressedWriteBuffer.h>
#include <IO/ReadBufferFromPocoSocket.h>
#include <IO/WriteBufferFromPocoSocket.h>
#include <IO/CompressionSettings.h>
#include <IO/copyData.h>
#include <DataStreams/AsynchronousBlockInputStream.h>
#include <DataStreams/NativeBlockInputStream.h>
#include <DataStreams/NativeBlockOutputStream.h>
#include <Interpreters/executeQuery.h>
#include <Interpreters/Quota.h>
#include <Interpreters/TablesStatus.h>
#include <Storages/StorageMemory.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Common/ClickHouseRevision.h>
#include <Common/Stopwatch.h>
#include <Common/ExternalTable.h>
#include "TCPHandler.h"
#include <Common/NetException.h>
#include <Common/config_version.h>
#include <ext/scope_guard.h>

View File

@ -137,7 +137,8 @@ public:
/// const_cast because we will sort the array
auto & array = const_cast<typename MaxIntersectionsData<PointType>::Array &>(this->data(place).value);
std::sort(array.begin(), array.end(), [](const auto & a, const auto & b) { return a.first < b.first; });
/// Sort by position; for equal position, sort by weight to get deterministic result.
std::sort(array.begin(), array.end());
for (const auto & point_weight : array)
{

View File

@ -19,6 +19,7 @@
#include <Common/CurrentMetrics.h>
#include <Common/DNSResolver.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/config_version.h>
#include <Interpreters/ClientInfo.h>
#include <Common/config.h>

View File

@ -7,6 +7,7 @@ const char * auto_config_build[]
"VERSION_FULL", "@VERSION_FULL@",
"VERSION_DESCRIBE", "@VERSION_DESCRIBE@",
"VERSION_GITHASH", "@VERSION_GITHASH@",
"VERSION_REVISION", "@VERSION_REVISION@",
"BUILD_DATE", "@BUILD_DATE@",
"BUILD_TYPE", "@CMAKE_BUILD_TYPE@",
"SYSTEM", "@CMAKE_SYSTEM@",

View File

@ -13,7 +13,31 @@
#cmakedefine VERSION_REVISION @VERSION_REVISION@
#endif
#cmakedefine VERSION_NAME "@VERSION_NAME@"
#define DBMS_NAME VERSION_NAME
#cmakedefine VERSION_MAJOR @VERSION_MAJOR@
#cmakedefine VERSION_MINOR @VERSION_MINOR@
#cmakedefine VERSION_PATCH @VERSION_PATCH@
#cmakedefine VERSION_STRING "@VERSION_STRING@"
#cmakedefine VERSION_FULL "@VERSION_FULL@"
#cmakedefine VERSION_DESCRIBE "@VERSION_DESCRIBE@"
#cmakedefine VERSION_GITHASH "@VERSION_GITHASH@"
#if defined(VERSION_MAJOR)
#define DBMS_VERSION_MAJOR VERSION_MAJOR
#else
#define DBMS_VERSION_MAJOR 0
#endif
#if defined(VERSION_MINOR)
#define DBMS_VERSION_MINOR VERSION_MINOR
#else
#define DBMS_VERSION_MINOR 0
#endif
#if defined(VERSION_PATCH)
#define DBMS_VERSION_PATCH VERSION_PATCH
#else
#define DBMS_VERSION_PATCH 0
#endif

View File

@ -1,9 +1,5 @@
#pragma once
#define DBMS_NAME "ClickHouse"
#define DBMS_VERSION_MAJOR 1
#define DBMS_VERSION_MINOR 1
#define DBMS_DEFAULT_HOST "localhost"
#define DBMS_DEFAULT_PORT 9000
#define DBMS_DEFAULT_SECURE_PORT 9440

View File

@ -323,22 +323,24 @@ void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, std::
if (current_key.empty()) /// The first key encountered.
{
setPrimaryKeyRef(current_key, current);
key_differs = true;
current_row_is_zero = true;
}
else
key_differs = next_key != current_key;
/// if there are enough rows and the last one is calculated completely
if (key_differs && merged_rows >= max_block_size)
return;
queue.pop();
if (key_differs)
{
/// Write the data for the previous group.
insertCurrentRowIfNeeded(merged_columns, false);
if (!current_key.empty())
/// Write the data for the previous group.
insertCurrentRowIfNeeded(merged_columns, false);
if (merged_rows >= max_block_size)
{
/// The block is now full and the last row is calculated completely.
current_key.reset();
return;
}
current_key.swap(next_key);
@ -375,6 +377,8 @@ void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, std::
current_row_is_zero = false;
}
queue.pop();
if (!current->isLast())
{
current->next();
@ -481,6 +485,9 @@ void SummingSortedBlockInputStream::addRow(SortCursor & cursor)
{
for (auto & desc : columns_to_aggregate)
{
if (!desc.created)
throw Exception("Logical error in SummingSortedBlockInputStream, there are no description", ErrorCodes::LOGICAL_ERROR);
if (desc.is_agg_func_type)
{
// desc.state is not used for AggregateFunction types
@ -489,9 +496,6 @@ void SummingSortedBlockInputStream::addRow(SortCursor & cursor)
}
else
{
if (!desc.created)
throw Exception("Logical error in SummingSortedBlockInputStream, there are no description", ErrorCodes::LOGICAL_ERROR);
// Specialized case for unary functions
if (desc.column_numbers.size() == 1)
{

View File

@ -4,6 +4,7 @@
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Formats/FormatSettings.h>
#include <Formats/IRowInputStream.h>
#include <common/logger_useful.h>
namespace DB
@ -25,7 +26,13 @@ public:
const FormatSettings & settings);
void readPrefix() override { row_input->readPrefix(); }
void readSuffix() override { row_input->readSuffix(); }
void readSuffix() override
{
Logger * log = &Logger::get("BlockInputStreamFromRowInputStream");
LOG_TRACE(log, "Skipped " << num_errors << " rows while reading the input stream");
row_input->readSuffix();
}
String getName() const override { return "BlockInputStreamFromRowInputStream"; }

View File

@ -15,6 +15,7 @@
#include <Common/UnicodeBar.h>
#include <Common/UTF8Helpers.h>
#include <Common/FieldVisitors.h>
#include <Common/config_version.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDate.h>
@ -1867,9 +1868,7 @@ public:
std::string FunctionVersion::getVersion() const
{
std::ostringstream os;
os << DBMS_VERSION_MAJOR << "." << DBMS_VERSION_MINOR << "." << ClickHouseRevision::get();
return os.str();
return VERSION_STRING;
}

View File

@ -77,6 +77,9 @@ public:
/// offset in bytes of the cursor from the beginning of the buffer
inline size_t offset() const { return size_t(pos - working_buffer.begin()); }
/// How many bytes are available for read/write
inline size_t available() const { return size_t(working_buffer.end() - pos); }
/** How many bytes have been read/written, counting those that are still in the buffer. */
size_t count() const
{

View File

@ -98,6 +98,22 @@ inline void writeBoolText(bool x, WriteBuffer & buf)
writeChar(x ? '1' : '0', buf);
}
template <typename T>
inline size_t writeFloatTextFastPath(T x, char * buffer, int len)
{
using Converter = DoubleConverter<false>;
double_conversion::StringBuilder builder{buffer, len};
bool result = false;
if constexpr (std::is_same_v<T, double>)
result = Converter::instance().ToShortest(x, &builder);
else
result = Converter::instance().ToShortestSingle(x, &builder);
if (!result)
throw Exception("Cannot print floating point number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER);
return builder.position();
}
template <typename T>
inline void writeFloatText(T x, WriteBuffer & buf)
@ -105,6 +121,11 @@ inline void writeFloatText(T x, WriteBuffer & buf)
static_assert(std::is_same_v<T, double> || std::is_same_v<T, float>, "Argument for writeFloatText must be float or double");
using Converter = DoubleConverter<false>;
if (likely(buf.available() >= Converter::MAX_REPRESENTATION_LENGTH))
{
buf.position() += writeFloatTextFastPath(x, buf.position(), Converter::MAX_REPRESENTATION_LENGTH);
return;
}
Converter::BufferType buffer;
double_conversion::StringBuilder builder{buffer, sizeof(buffer)};
@ -118,7 +139,6 @@ inline void writeFloatText(T x, WriteBuffer & buf)
if (!result)
throw Exception("Cannot print floating point number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER);
/// TODO Excessive copy. Use optimistic path if buffer have enough bytes.
buf.write(buffer, builder.position());
}

View File

@ -6,6 +6,7 @@
#include <Core/Defines.h>
#include <Common/getFQDNOrHostName.h>
#include <Common/ClickHouseRevision.h>
#include <Common/config_version.h>
#include <port/unistd.h>

View File

@ -357,11 +357,10 @@ void LogicalExpressionsOptimizer::fixBrokenOrExpressions()
for (auto & parent : parents)
{
parent->children.push_back(operands[0]);
auto first_erased = std::remove_if(parent->children.begin(), parent->children.end(),
[or_function](const ASTPtr & ptr) { return ptr.get() == or_function; });
parent->children.erase(first_erased, parent->children.end());
// The order of children matters if or is children of some function, e.g. minus
std::replace_if(parent->children.begin(), parent->children.end(),
[or_function](const ASTPtr & ptr) { return ptr.get() == or_function; },
operands[0] );
}
/// If the OR node was the root of the WHERE, PREWHERE, or HAVING expression, then update this root.

View File

@ -1,3 +1,5 @@
10000
10000
1 6 3 3
1 6 3 3
1 6 [3,2]

View File

@ -1,6 +1,33 @@
drop table if exists test.summing_merge_tree_aggregate_function;
drop table if exists test.summing_merge_tree_null;
---- partition merge
create table test.summing_merge_tree_aggregate_function (
d Date,
k UInt64,
u AggregateFunction(uniq, UInt64)
) engine=SummingMergeTree(d, k, 1);
insert into test.summing_merge_tree_aggregate_function
select today() as d,
number as k,
uniqState(toUInt64(number % 500))
from numbers(5000)
group by d, k;
insert into test.summing_merge_tree_aggregate_function
select today() as d,
number + 5000 as k,
uniqState(toUInt64(number % 500))
from numbers(5000)
group by d, k;
select count() from test.summing_merge_tree_aggregate_function;
optimize table test.summing_merge_tree_aggregate_function;
select count() from test.summing_merge_tree_aggregate_function;
drop table test.summing_merge_tree_aggregate_function;
---- sum + uniq + uniqExact
create table test.summing_merge_tree_aggregate_function (
d materialized today(),

View File

@ -0,0 +1,2 @@
1
0

View File

@ -0,0 +1,8 @@
drop table if exists test.orin_test;
create table test.orin_test (c1 Int32) engine=Memory;
insert into test.orin_test values(1), (100);
select minus(c1 = 1 or c1=2 or c1 =3, c1=5) from test.orin_test;
drop table test.orin_test;

View File

@ -0,0 +1,2 @@
1
1

View File

@ -0,0 +1,14 @@
DROP TABLE IF EXISTS test.test1;
DROP TABLE IF EXISTS test.test2;
CREATE TABLE test.test1(start Integer, end Integer) engine = Memory;
CREATE TABLE test.test2(start Integer, end Integer) engine = Memory;
INSERT INTO test.test1(start,end) VALUES (1,3),(3,5);
INSERT INTO test.test2(start,end) VALUES (3,5),(1,3);
SELECT maxIntersections(start,end) from test.test1;
SELECT maxIntersections(start,end) from test.test2;
DROP TABLE test.test1;
DROP TABLE test.test2;

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (1.1.54394) unstable; urgency=low
clickhouse (1.1.54398) unstable; urgency=low
* Modified source code
-- <robot-metrika-test@yandex-team.ru> Thu, 12 Jul 2018 16:25:17 +0300
-- <robot-metrika-test@yandex-team.ru> Tue, 17 Jul 2018 20:04:13 +0300

2
debian/changelog.in vendored
View File

@ -1,4 +1,4 @@
clickhouse (1.1.@REVISION@) unstable; urgency=low
clickhouse (@VERSION_STRING@) unstable; urgency=low
* Modified source code

View File

@ -51,7 +51,7 @@ The history is written to `~/.clickhouse-client-history`.
By default, the format used is PrettyCompact. You can change the format in the FORMAT clause of the query, or by specifying `\G` at the end of the query, using the `--format` or `--vertical` argument in the command line, or using the client configuration file.
To exit the client, press Ctrl+D (or Ctrl+C), or enter one of the following instead of a query:"exit", "quit", "logout", "учше", "йгше", "дщпщге", "exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж", "q", "й", "q", "Q", ":q", "й", "Й", "Жй"
To exit the client, press Ctrl+D (or Ctrl+C), or enter one of the following instead of a query: "exit", "quit", "logout", "exit;", "quit;", "logout;", "q", "Q", ":q"
When processing a query, the client shows:

View File

@ -61,6 +61,7 @@
#include <IO/WriteHelpers.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Common/ClickHouseRevision.h>
#include <Common/config_version.h>
#include <daemon/OwnPatternFormatter.h>
#include <Poco/Net/RemoteSyslogChannel.h>
@ -1083,7 +1084,7 @@ void BaseDaemon::initialize(Application & self)
void BaseDaemon::logRevision() const
{
Logger::root().information("Starting daemon with revision " + Poco::NumberFormatter::format(ClickHouseRevision::get()));
Logger::root().information("Starting " + std::string{VERSION_FULL} + " with revision " + Poco::NumberFormatter::format(ClickHouseRevision::get()));
}
/// Makes server shutdown if at least one Poco::Task have failed.

29
release
View File

@ -10,6 +10,16 @@
# Clang6 without internal compiler (for low memory arm64):
# env DIST=bionic DISABLE_PARALLEL=1 EXTRAPACKAGES="clang-6.0 libstdc++-8-dev" DEB_CC=clang-6.0 DEB_CXX=clang++-6.0 CMAKE_FLAGS=" -DNO_WERROR=1 " ./release
# Version increment:
# Default release: 18.1.2 -> 18.2.0:
# ./release --version
# or
# ./release --version minor
# Bugfix release (only with small patches to previous release): 18.1.2 -> 18.1.3:
# ./release --version patch
# Do this once per year: 18.1.2 -> 19.0.0:
# ./release --version major
set -e
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
@ -26,7 +36,7 @@ DEBUILD_NOSIGN_OPTIONS="-us -uc"
DEBUILD_NODEPS_OPTIONS="-d"
USE_PBUILDER=1
if [ -z "$REVISION" ] ; then
if [ -z "$VERSION_STRING" ] ; then
get_revision_author
fi
@ -40,12 +50,12 @@ do
DEBUILD_NODEPS_OPTIONS=""
shift
elif [[ $1 == '--version' ]]; then
gen_revision_author
git push
gen_revision_author $2
exit 0
elif [[ $1 == '--head' ]]; then
REVISION=`git rev-parse HEAD`
shift
# who use this?
#elif [[ $1 == '--head' ]]; then
# VERSION_STRING=`git rev-parse HEAD`
# shift
elif [[ $1 == '--pbuilder' ]]; then
# Default
shift
@ -62,7 +72,6 @@ do
fi
done
# Build options
if [ -n "$SANITIZER" ]
then
@ -90,10 +99,10 @@ CMAKE_FLAGS=" $LIBTCMALLOC_OPTS -DENABLE_EMBEDDED_COMPILER=1 $CMAKE_FLAGS"
export CMAKE_FLAGS
export EXTRAPACKAGES
REVISION+=$VERSION_POSTFIX
echo -e "\nCurrent revision is $REVISION"
VERSION_STRING+=$VERSION_POSTFIX
echo -e "\nCurrent version is $VERSION_STRING"
gen_changelog "$REVISION" "" "$AUTHOR" ""
gen_changelog "$VERSION_STRING" "" "$AUTHOR" ""
if [ -z "$USE_PBUILDER" ] ; then
# Build (only binary packages).

View File

@ -1,8 +1,23 @@
set +e
function get_revision {
function gen_version_string {
if [ -n "$TEST" ]; then
VERSION_STRING="$VERSION_MAJOR.$VERSION_MINOR.$VERSION_PATCH.$VERSION_REVISION"
else
VERSION_STRING="$VERSION_MAJOR.$VERSION_MINOR.$VERSION_PATCH"
fi
}
function get_version {
BASEDIR=$(dirname "${BASH_SOURCE[0]}")
grep "set(VERSION_REVISION" ${BASEDIR}/dbms/cmake/version.cmake | sed 's/^.*VERSION_REVISION \(.*\))$/\1/'
VERSION_REVISION=`grep "set(VERSION_REVISION" ${BASEDIR}/dbms/cmake/version.cmake | sed 's/^.*VERSION_REVISION \(.*\)$/\1/' | sed 's/[) ].*//'`
VERSION_MAJOR=`grep "set(VERSION_MAJOR" ${BASEDIR}/dbms/cmake/version.cmake | sed 's/^.*VERSION_MAJOR \(.*\)/\1/' | sed 's/[) ].*//'`
VERSION_MINOR=`grep "set(VERSION_MINOR" ${BASEDIR}/dbms/cmake/version.cmake | sed 's/^.*VERSION_MINOR \(.*\)/\1/' | sed 's/[) ].*//'`
VERSION_PATCH=`grep "set(VERSION_PATCH" ${BASEDIR}/dbms/cmake/version.cmake | sed 's/^.*VERSION_PATCH \(.*\)/\1/' | sed 's/[) ].*//'`
VERSION_PREFIX="${VERSION_PREFIX:-v}"
VERSION_POSTFIX="${VERSION_POSTFIX:--testing}"
gen_version_string
}
function get_author {
@ -13,9 +28,8 @@ function get_author {
# Generate revision number.
# set environment variables REVISION, AUTHOR
function gen_revision_author {
REVISION=$(get_revision)
VERSION_PREFIX="${VERSION_PREFIX:-v1.1.}"
VERSION_POSTFIX="${VERSION_POSTFIX:--testing}"
TYPE=$1
get_version
if [[ $STANDALONE != 'yes' ]]; then
@ -26,21 +40,40 @@ function gen_revision_author {
max_attempts=1000
while [ $succeeded -eq 0 ] && [ $attempts -le $max_attempts ]; do
attempts=$(($attempts + 1))
REVISION=$(($REVISION + 1))
git_tag_grep=`git tag | grep "$VERSION_PREFIX$REVISION$VERSION_POSTFIX"`
if [ "$TYPE" == "major" ]; then
VERSION_REVISION=$(($VERSION_REVISION + 1))
VERSION_MAJOR=$(($VERSION_MAJOR + 1))
VERSION_MINOR=0
VERSION_PATCH=0
elif [ "$TYPE" == "minor" ] || [ "$TYPE" == "" ]; then
VERSION_REVISION=$(($VERSION_REVISION + 1))
VERSION_MINOR=$(($VERSION_MINOR + 1))
VERSION_PATCH=0
elif [ "$TYPE" == "patch" ] || [ "$TYPE" == "bugfix" ]; then
# VERSION_REVISION not incremented.
VERSION_PATCH=$(($VERSION_PATCH + 1))
else
echo "Unknown version type $TYPE"
exit 1
fi
gen_version_string
git_tag_grep=`git tag | grep "$VERSION_PREFIX$VERSION_STRING$VERSION_POSTFIX"`
if [ "$git_tag_grep" == "" ]; then
succeeded=1
fi
done
if [ $succeeded -eq 0 ]; then
echo "Fail to create revision up to $REVISION"
echo "Fail to create revision up to $VERSION_REVISION"
exit 1
fi
auto_message="Auto version update to"
git_log_grep=`git log --oneline --max-count=1 | grep "$auto_message"`
if [ "$git_log_grep" == "" ]; then
tag="$VERSION_PREFIX$REVISION$VERSION_POSTFIX"
tag="$VERSION_PREFIX$VERSION_STRING$VERSION_POSTFIX"
# First tag for correct git describe
echo -e "\nTrying to create tag: $tag"
@ -48,11 +81,18 @@ function gen_revision_author {
git_describe=`git describe`
git_hash=`git rev-parse HEAD`
sed -i -- "s/VERSION_REVISION .*)/VERSION_REVISION $REVISION)/g;s/VERSION_DESCRIBE .*)/VERSION_DESCRIBE $git_describe)/g;s/VERSION_GITHASH .*)/VERSION_GITHASH $git_hash)/g;" dbms/cmake/version.cmake
sed -i -e "s/set(VERSION_REVISION [^) ]*/set(VERSION_REVISION $VERSION_REVISION/g;" \
-e "s/set(VERSION_DESCRIBE [^) ]*/set(VERSION_DESCRIBE $git_describe/g;" \
-e "s/set(VERSION_GITHASH [^) ]*/set(VERSION_GITHASH $git_hash/g;" \
-e "s/set(VERSION_MAJOR [^) ]*/set(VERSION_MAJOR $VERSION_MAJOR/g;" \
-e "s/set(VERSION_MINOR [^) ]*/set(VERSION_MINOR $VERSION_MINOR/g;" \
-e "s/set(VERSION_PATCH [^) ]*/set(VERSION_PATCH $VERSION_PATCH/g;" \
-e "s/set(VERSION_STRING [^) ]*/set(VERSION_STRING $VERSION_STRING/g;" \
dbms/cmake/version.cmake
gen_changelog "$REVISION" "" "$AUTHOR" ""
git commit -m "$auto_message [$REVISION]" dbms/cmake/version.cmake debian/changelog
#git push
gen_changelog "$VERSION_STRING" "" "$AUTHOR" ""
git commit -m "$auto_message [$VERSION_STRING] [$VERSION_REVISION]" dbms/cmake/version.cmake debian/changelog
git push
# Second tag for correct version information in version.cmake inside tag
if git tag --force -a "$tag" -m "$tag"
@ -68,34 +108,29 @@ function gen_revision_author {
fi
else
REVISION=$(get_revision)
echo reusing old version $REVISION
get_version
echo reusing old version $VERSION_STRING
fi
fi
AUTHOR=$(git config --get user.name || echo ${USER})
export REVISION
export AUTHOR
}
function get_revision_author {
REVISION=$(get_revision)
get_version
AUTHOR=$(get_author)
export REVISION
export AUTHOR
}
# Generate changelog from changelog.in.
# changes
# programs/CMakeLists.txt
# dbms/src/CMakeLists.txt
function gen_changelog {
REVISION="$1"
VERSION_STRING="$1"
CHDATE="$2"
AUTHOR="$3"
CHLOG="$4"
if [ -z "REVISION" ] ; then
if [ -z "$VERSION_STRING" ] ; then
get_revision_author
fi
@ -108,7 +143,7 @@ function gen_changelog {
fi
sed \
-e "s/[@]REVISION[@]/$REVISION/g" \
-e "s/[@]VERSION_STRING[@]/$VERSION_STRING/g" \
-e "s/[@]DATE[@]/$CHDATE/g" \
-e "s/[@]AUTHOR[@]/$AUTHOR/g" \
-e "s/[@]EMAIL[@]/$(whoami)@yandex-team.ru/g" \