Skip to content

Commit

Permalink
Merge remote-tracking branch 'blessed/master' into parallel_replicas_…
Browse files Browse the repository at this point in the history
…cte_2
  • Loading branch information
Algunenano committed Mar 1, 2024
2 parents 77752a6 + 4fd64a6 commit 0dcaf3f
Show file tree
Hide file tree
Showing 53 changed files with 340 additions and 263 deletions.
1 change: 1 addition & 0 deletions CHANGELOG.md
Expand Up @@ -31,6 +31,7 @@
* Support single-argument version for the `merge` table function, as `merge(['db_name', ] 'tables_regexp')`. [#60372](https://github.com/ClickHouse/ClickHouse/pull/60372) ([豪肥肥](https://github.com/HowePa)).
* Support negative positional arguments. Closes [#57736](https://github.com/ClickHouse/ClickHouse/issues/57736). [#58292](https://github.com/ClickHouse/ClickHouse/pull/58292) ([flynn](https://github.com/ucasfl)).
* Support specifying a set of permitted users for specific S3 settings in config using `user` key. [#60144](https://github.com/ClickHouse/ClickHouse/pull/60144) ([Antonio Andelic](https://github.com/antonio2368)).
* Added table function `mergeTreeIndex`. It represents the contents of index and marks files of `MergeTree` tables. It can be used for introspection. Syntax: `mergeTreeIndex(database, table, [with_marks = true])` where `database.table` is an existing table with `MergeTree` engine. [#58140](https://github.com/ClickHouse/ClickHouse/pull/58140) ([Anton Popov](https://github.com/CurtizJ)).

#### Experimental Feature
* Add function `variantType` that returns Enum with variant type name for each row. [#59398](https://github.com/ClickHouse/ClickHouse/pull/59398) ([Kruglov Pavel](https://github.com/Avogar)).
Expand Down
60 changes: 14 additions & 46 deletions docs/en/sql-reference/functions/date-time-functions.md
Expand Up @@ -394,7 +394,8 @@ Result:

## toYear

Returns the year component (AD) of a date or date with time.
Converts a date or date with time to the year number (AD) as `UInt16` value.


**Syntax**

Expand Down Expand Up @@ -430,7 +431,7 @@ Result:

## toQuarter

Returns the quarter (1-4) of a date or date with time.
Converts a date or date with time to the quarter number (1-4) as `UInt8` value.

**Syntax**

Expand Down Expand Up @@ -464,9 +465,10 @@ Result:
└──────────────────────────────────────────────┘
```


## toMonth

Returns the month component (1-12) of a date or date with time.
Converts a date or date with time to the month number (1-12) as `UInt8` value.

**Syntax**

Expand Down Expand Up @@ -502,7 +504,7 @@ Result:

## toDayOfYear

Returns the number of the day within the year (1-366) of a date or date with time.
Converts a date or date with time to the number of the day of the year (1-366) as `UInt16` value.

**Syntax**

Expand Down Expand Up @@ -538,7 +540,7 @@ Result:

## toDayOfMonth

Returns the number of the day within the month (1-31) of a date or date with time.
Converts a date or date with time to the number of the day in the month (1-31) as `UInt8` value.

**Syntax**

Expand Down Expand Up @@ -574,7 +576,7 @@ Result:

## toDayOfWeek

Returns the number of the day within the week of a date or date with time.
Converts a date or date with time to the number of the day in the week as `UInt8` value.

The two-argument form of `toDayOfWeek()` enables you to specify whether the week starts on Monday or Sunday, and whether the return value should be in the range from 0 to 6 or 1 to 7. If the mode argument is omitted, the default mode is 0. The time zone of the date can be specified as the third argument.

Expand Down Expand Up @@ -625,7 +627,7 @@ Result:

## toHour

Returns the hour component (0-24) of a date with time.
Converts a date with time to the number of the hour in 24-hour time (0-23) as `UInt8` value.

Assumes that if clocks are moved ahead, it is by one hour and occurs at 2 a.m., and if clocks are moved back, it is by one hour and occurs at 3 a.m. (which is not always exactly when it occurs - it depends on the timezone).

Expand All @@ -639,7 +641,7 @@ Alias: `HOUR`

**Arguments**

- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)

**Returned value**

Expand All @@ -663,7 +665,7 @@ Result:

## toMinute

Returns the minute component (0-59) a date with time.
Converts a date with time to the number of the minute of the hour (0-59) as `UInt8` value.

**Syntax**

Expand All @@ -675,7 +677,7 @@ Alias: `MINUTE`

**Arguments**

- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)

**Returned value**

Expand All @@ -699,7 +701,7 @@ Result:

## toSecond

Returns the second component (0-59) of a date with time. Leap seconds are not considered.
Converts a date with time to the second in the minute (0-59) as `UInt8` value. Leap seconds are not considered.

**Syntax**

Expand All @@ -711,7 +713,7 @@ Alias: `SECOND`

**Arguments**

- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)

**Returned value**

Expand All @@ -733,40 +735,6 @@ Result:
└─────────────────────────────────────────────┘
```

## toMillisecond

Returns the millisecond component (0-999) of a date with time.

**Syntax**

```sql
toMillisecond(value)
```

*Arguments**

- `value` - [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)

Alias: `MILLISECOND`

```sql
SELECT toMillisecond(toDateTime64('2023-04-21 10:20:30.456', 3))
```

Result:

```response
┌──toMillisecond(toDateTime64('2023-04-21 10:20:30.456', 3))─┐
│ 456 │
└────────────────────────────────────────────────────────────┘
```

**Returned value**

- The millisecond in the minute (0 - 59) of the given date/time

Type: `UInt16`

## toUnixTimestamp

Converts a string, a date or a date with time to the [Unix Timestamp](https://en.wikipedia.org/wiki/Unix_time) in `UInt32` representation.
Expand Down
1 change: 1 addition & 0 deletions programs/keeper-converter/KeeperConverter.cpp
Expand Up @@ -4,6 +4,7 @@
#include <Coordination/CoordinationSettings.h>
#include <Coordination/KeeperSnapshotManager.h>
#include <Coordination/ZooKeeperDataReader.h>
#include <Coordination/KeeperContext.h>
#include <Common/TerminalSize.h>
#include <Poco/ConsoleChannel.h>
#include <Poco/AutoPtr.h>
Expand Down
40 changes: 8 additions & 32 deletions src/Common/DateLUTImpl.h
Expand Up @@ -3,13 +3,13 @@
#include <base/DayNum.h>
#include <base/defines.h>
#include <base/types.h>
#include <Core/DecimalFunctions.h>

#include <ctime>
#include <cassert>
#include <string>
#include <type_traits>


#define DATE_SECONDS_PER_DAY 86400 /// Number of seconds in a day, 60 * 60 * 24

#define DATE_LUT_MIN_YEAR 1900 /// 1900 since majority of financial organizations consider 1900 as an initial year.
Expand Down Expand Up @@ -280,9 +280,9 @@ class DateLUTImpl
static_assert(std::is_integral_v<DateOrTime> && std::is_integral_v<Divisor>);
assert(divisor > 0);

if (offset_is_whole_number_of_hours_during_epoch) [[likely]]
if (likely(offset_is_whole_number_of_hours_during_epoch))
{
if (x >= 0) [[likely]]
if (likely(x >= 0))
return static_cast<DateOrTime>(x / divisor * divisor);

/// Integer division for negative numbers rounds them towards zero (up).
Expand Down Expand Up @@ -576,10 +576,10 @@ class DateLUTImpl

unsigned toSecond(Time t) const
{
if (offset_is_whole_number_of_minutes_during_epoch) [[likely]]
if (likely(offset_is_whole_number_of_minutes_during_epoch))
{
Time res = t % 60;
if (res >= 0) [[likely]]
if (likely(res >= 0))
return static_cast<unsigned>(res);
return static_cast<unsigned>(res) + 60;
}
Expand All @@ -593,30 +593,6 @@ class DateLUTImpl
return time % 60;
}

template <typename DateOrTime>
unsigned toMillisecond(const DateOrTime & datetime, Int64 scale_multiplier) const
{
constexpr Int64 millisecond_multiplier = 1'000;
constexpr Int64 microsecond_multiplier = 1'000 * millisecond_multiplier;
constexpr Int64 divider = microsecond_multiplier / millisecond_multiplier;

auto components = DB::DecimalUtils::splitWithScaleMultiplier(datetime, scale_multiplier);

if (datetime.value < 0 && components.fractional)
{
components.fractional = scale_multiplier + (components.whole ? Int64(-1) : Int64(1)) * components.fractional;
--components.whole;
}
Int64 fractional = components.fractional;
if (scale_multiplier > microsecond_multiplier)
fractional = fractional / (scale_multiplier / microsecond_multiplier);
else if (scale_multiplier < microsecond_multiplier)
fractional = fractional * (microsecond_multiplier / scale_multiplier);

UInt16 millisecond = static_cast<UInt16>(fractional / divider);
return millisecond;
}

unsigned toMinute(Time t) const
{
if (t >= 0 && offset_is_whole_number_of_hours_during_epoch)
Expand Down Expand Up @@ -1146,9 +1122,9 @@ class DateLUTImpl
DateOrTime toStartOfMinuteInterval(DateOrTime t, UInt64 minutes) const
{
Int64 divisor = 60 * minutes;
if (offset_is_whole_number_of_minutes_during_epoch) [[likely]]
if (likely(offset_is_whole_number_of_minutes_during_epoch))
{
if (t >= 0) [[likely]]
if (likely(t >= 0))
return static_cast<DateOrTime>(t / divisor * divisor);
return static_cast<DateOrTime>((t + 1 - divisor) / divisor * divisor);
}
Expand Down Expand Up @@ -1363,7 +1339,7 @@ class DateLUTImpl

UInt8 saturateDayOfMonth(Int16 year, UInt8 month, UInt8 day_of_month) const
{
if (day_of_month <= 28) [[likely]]
if (likely(day_of_month <= 28))
return day_of_month;

UInt8 days_in_month = daysInMonth(year, month);
Expand Down
2 changes: 2 additions & 0 deletions src/Coordination/KeeperDispatcher.cpp
Expand Up @@ -13,6 +13,8 @@
#include <Common/logger_useful.h>
#include <Common/formatReadable.h>

#include <Disks/IDisk.h>

#include <atomic>
#include <future>
#include <chrono>
Expand Down
44 changes: 35 additions & 9 deletions src/Coordination/KeeperSnapshotManager.cpp
@@ -1,24 +1,24 @@
#include <filesystem>
#include <memory>
#include <Compression/CompressedReadBuffer.h>
#include <Compression/CompressedWriteBuffer.h>
#include <Coordination/CoordinationSettings.h>
#include <Coordination/KeeperCommon.h>
#include <Coordination/KeeperConstants.h>
#include <Coordination/KeeperContext.h>
#include <Coordination/KeeperSnapshotManager.h>
#include <Coordination/ReadBufferFromNuraftBuffer.h>
#include <Coordination/WriteBufferFromNuraftBuffer.h>
#include <Coordination/CoordinationSettings.h>
#include <Core/Field.h>
#include <Disks/DiskLocal.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteHelpers.h>
#include <IO/copyData.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <filesystem>
#include <memory>
#include <Common/logger_useful.h>
#include <Coordination/KeeperContext.h>
#include <Coordination/KeeperCommon.h>
#include <Coordination/KeeperConstants.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include <Core/Field.h>
#include <Disks/DiskLocal.h>

namespace DB
{
Expand Down Expand Up @@ -862,4 +862,30 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor
return {snapshot_file_name, disk};
}

size_t KeeperSnapshotManager::getLatestSnapshotIndex() const
{
if (!existing_snapshots.empty())
return existing_snapshots.rbegin()->first;
return 0;
}

SnapshotFileInfo KeeperSnapshotManager::getLatestSnapshotInfo() const
{
if (!existing_snapshots.empty())
{
const auto & [path, disk] = existing_snapshots.at(getLatestSnapshotIndex());

try
{
if (disk->exists(path))
return {path, disk};
}
catch (...)
{
tryLogCurrentException(log);
}
}
return {"", nullptr};
}

}
42 changes: 12 additions & 30 deletions src/Coordination/KeeperSnapshotManager.h
@@ -1,12 +1,6 @@
#pragma once
#include <filesystem>
#include <system_error>
#include <Coordination/KeeperStorage.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h>
#include <libnuraft/nuraft.hxx>
#include <Coordination/KeeperContext.h>
#include <Disks/IDisk.h>

namespace DB
{
Expand All @@ -16,6 +10,15 @@ using SnapshotMetadataPtr = std::shared_ptr<SnapshotMetadata>;
using ClusterConfig = nuraft::cluster_config;
using ClusterConfigPtr = nuraft::ptr<ClusterConfig>;

class WriteBuffer;
class ReadBuffer;

class KeeperContext;
using KeeperContextPtr = std::shared_ptr<KeeperContext>;

class IDisk;
using DiskPtr = std::shared_ptr<IDisk>;

enum SnapshotVersion : uint8_t
{
V0 = 0,
Expand Down Expand Up @@ -139,30 +142,9 @@ class KeeperSnapshotManager
size_t totalSnapshots() const { return existing_snapshots.size(); }

/// The most fresh snapshot log index we have
size_t getLatestSnapshotIndex() const
{
if (!existing_snapshots.empty())
return existing_snapshots.rbegin()->first;
return 0;
}

SnapshotFileInfo getLatestSnapshotInfo() const
{
if (!existing_snapshots.empty())
{
const auto & [path, disk] = existing_snapshots.at(getLatestSnapshotIndex());

try
{
if (disk->exists(path))
return {path, disk};
}
catch (...)
{
}
}
return {"", nullptr};
}
size_t getLatestSnapshotIndex() const;

SnapshotFileInfo getLatestSnapshotInfo() const;

private:
void removeOutdatedSnapshotsIfNeeded();
Expand Down

0 comments on commit 0dcaf3f

Please sign in to comment.