mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
Fix test and add one
This commit is contained in:
parent
fefce49c8f
commit
dd37370aab
@ -2834,4 +2834,16 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String &
|
||||
LOG_DEBUG(log, "Freezed " << parts_processed << " parts");
|
||||
}
|
||||
|
||||
bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const
|
||||
{
|
||||
if (!settings.enable_mixed_granularity_parts || settings.index_granularity_bytes == 0)
|
||||
{
|
||||
if (!canUseAdaptiveGranularity() && src_part->index_granularity_info.is_adaptive)
|
||||
return false;
|
||||
if (canUseAdaptiveGranularity() && !src_part->index_granularity_info.is_adaptive)
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -580,6 +580,7 @@ public:
|
||||
(settings.enable_mixed_granularity_parts || !has_non_adaptive_index_granularity_parts);
|
||||
}
|
||||
|
||||
|
||||
MergeTreeDataFormatVersion format_version;
|
||||
|
||||
Context global_context;
|
||||
@ -798,6 +799,9 @@ protected:
|
||||
/// Common part for |freezePartition()| and |freezeAll()|.
|
||||
using MatcherFn = std::function<bool(const DataPartPtr &)>;
|
||||
void freezePartitionsByMatcher(MatcherFn matcher, const String & with_name, const Context & context);
|
||||
|
||||
bool canReplacePartition(const DataPartPtr & data_part) const;
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1057,6 +1057,11 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con
|
||||
|
||||
for (const DataPartPtr & src_part : src_parts)
|
||||
{
|
||||
if (!canReplacePartition(src_part))
|
||||
throw Exception(
|
||||
"Cannot replace partition '" + partition_id + "' because part '" + src_part->name + "' has inconsistent granularity with table",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// This will generate unique name in scope of current server process.
|
||||
Int64 temp_index = insert_increment.get();
|
||||
MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level);
|
||||
|
@ -4787,6 +4787,12 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
|
||||
/// Save deduplication block ids with special prefix replace_partition
|
||||
|
||||
auto & src_part = src_all_parts[i];
|
||||
|
||||
if (!canReplacePartition(src_part))
|
||||
throw Exception(
|
||||
"Cannot replace partition '" + partition_id + "' because part '" + src_part->name + "' has inconsistent granularity with table",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
String hash_hex = src_part->checksums.getTotalChecksumHex();
|
||||
String block_id_path = replace ? "" : (zookeeper_path + "/blocks/" + partition_id + "_replace_from_" + hash_hex);
|
||||
|
||||
|
@ -13,7 +13,7 @@ chl="$CLICKHOUSE_CLIENT -q"
|
||||
ch_dir=`${CLICKHOUSE_EXTRACT_CONFIG} -k path`
|
||||
|
||||
$chl "DROP TABLE IF EXISTS test.partition_428"
|
||||
$chl "CREATE TABLE test.partition_428 (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) ENGINE = MergeTree(p, k, 1)"
|
||||
$chl "CREATE TABLE test.partition_428 (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0"
|
||||
$chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(31), 1)"
|
||||
$chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(1), 2)"
|
||||
|
||||
@ -51,7 +51,7 @@ $chl "DROP TABLE test.partition_428"
|
||||
# Test 2. Simple test
|
||||
|
||||
$chl "drop table if exists test.partition_428"
|
||||
$chl "create table test.partition_428 (date MATERIALIZED toDate(0), x UInt64, sample_key MATERIALIZED intHash64(x)) ENGINE=MergeTree(date,sample_key,(date,x,sample_key),8192)"
|
||||
$chl "create table test.partition_428 (date MATERIALIZED toDate(0), x UInt64, sample_key MATERIALIZED intHash64(x)) ENGINE=MergeTree PARTITION BY date SAMPLE BY sample_key ORDER BY (date,x,sample_key) SETTINGS index_granularity=8192, index_granularity_bytes=0"
|
||||
$chl "insert into test.partition_428 ( x ) VALUES ( now() )"
|
||||
$chl "insert into test.partition_428 ( x ) VALUES ( now()+1 )"
|
||||
$chl "alter table test.partition_428 detach partition 197001"
|
||||
|
@ -0,0 +1,3 @@
|
||||
8873898
|
||||
8873899
|
||||
8873899
|
@ -0,0 +1,53 @@
|
||||
DROP TABLE IF EXISTS mixed_granularity_table;
|
||||
|
||||
CREATE TABLE mixed_granularity_table (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, enable_mixed_granularity_parts=1; -- same with hits, but enabled mixed granularity
|
||||
|
||||
INSERT INTO mixed_granularity_table SELECT * FROM test.hits LIMIT 10;
|
||||
|
||||
ALTER TABLE mixed_granularity_table REPLACE PARTITION 201403 FROM test.hits;
|
||||
|
||||
SELECT COUNT() FROM mixed_granularity_table;
|
||||
|
||||
INSERT INTO mixed_granularity_table SELECT * FROM test.hits LIMIT 1;
|
||||
|
||||
SELECT COUNT() FROM mixed_granularity_table;
|
||||
|
||||
OPTIMIZE TABLE mixed_granularity_table FINAL;
|
||||
|
||||
SELECT COUNT() FROM mixed_granularity_table;
|
||||
|
||||
-- check strange cases when we try to replace parts from another tables but with different granularity settings
|
||||
|
||||
DROP TABLE IF EXISTS non_mixed_granularity_adaptive_table;
|
||||
|
||||
CREATE TABLE non_mixed_granularity_adaptive_table AS test.hits;
|
||||
|
||||
INSERT INTO non_mixed_granularity_adaptive_table SELECT * FROM test.hits LIMIT 10;
|
||||
|
||||
ALTER TABLE non_mixed_granularity_adaptive_table REPLACE PARTITION 201403 FROM test.hits; -- { serverError 49 }
|
||||
|
||||
DROP TABLE IF EXISTS non_mixed_granularity_adaptive_table;
|
||||
|
||||
DROP TABLE IF EXISTS non_mixed_granularity_non_adaptive_table;
|
||||
|
||||
CREATE TABLE non_mixed_granularity_non_adaptive_table (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, index_granularity_bytes=0; -- same with hits, but enabled mixed granularity and fixed_granularity
|
||||
|
||||
INSERT INTO non_mixed_granularity_non_adaptive_table SELECT * FROM test.hits LIMIT 10;
|
||||
|
||||
-- after optimize mixed_granularity_table will have .mrk2 parts
|
||||
ALTER TABLE non_mixed_granularity_non_adaptive_table REPLACE PARTITION 201403 FROM mixed_granularity_table; -- { serverError 49 }
|
||||
|
||||
DROP TABLE IF EXISTS non_mixed_granularity_non_adaptive_table;
|
||||
|
||||
|
||||
DROP TABLE IF EXISTS mixed_granularity_strictly_non_adaptive_table;
|
||||
|
||||
CREATE TABLE mixed_granularity_strictly_non_adaptive_table (`WatchID` UInt64, `JavaEnable` UInt8, `Title` String, `GoodEvent` Int16, `EventTime` DateTime, `EventDate` Date, `CounterID` UInt32, `ClientIP` UInt32, `ClientIP6` FixedString(16), `RegionID` UInt32, `UserID` UInt64, `CounterClass` Int8, `OS` UInt8, `UserAgent` UInt8, `URL` String, `Referer` String, `URLDomain` String, `RefererDomain` String, `Refresh` UInt8, `IsRobot` UInt8, `RefererCategories` Array(UInt16), `URLCategories` Array(UInt16), `URLRegions` Array(UInt32), `RefererRegions` Array(UInt32), `ResolutionWidth` UInt16, `ResolutionHeight` UInt16, `ResolutionDepth` UInt8, `FlashMajor` UInt8, `FlashMinor` UInt8, `FlashMinor2` String, `NetMajor` UInt8, `NetMinor` UInt8, `UserAgentMajor` UInt16, `UserAgentMinor` FixedString(2), `CookieEnable` UInt8, `JavascriptEnable` UInt8, `IsMobile` UInt8, `MobilePhone` UInt8, `MobilePhoneModel` String, `Params` String, `IPNetworkID` UInt32, `TraficSourceID` Int8, `SearchEngineID` UInt16, `SearchPhrase` String, `AdvEngineID` UInt8, `IsArtifical` UInt8, `WindowClientWidth` UInt16, `WindowClientHeight` UInt16, `ClientTimeZone` Int16, `ClientEventTime` DateTime, `SilverlightVersion1` UInt8, `SilverlightVersion2` UInt8, `SilverlightVersion3` UInt32, `SilverlightVersion4` UInt16, `PageCharset` String, `CodeVersion` UInt32, `IsLink` UInt8, `IsDownload` UInt8, `IsNotBounce` UInt8, `FUniqID` UInt64, `HID` UInt32, `IsOldCounter` UInt8, `IsEvent` UInt8, `IsParameter` UInt8, `DontCountHits` UInt8, `WithHash` UInt8, `HitColor` FixedString(1), `UTCEventTime` DateTime, `Age` UInt8, `Sex` UInt8, `Income` UInt8, `Interests` UInt16, `Robotness` UInt8, `GeneralInterests` Array(UInt16), `RemoteIP` UInt32, `RemoteIP6` FixedString(16), `WindowName` Int32, `OpenerName` Int32, `HistoryLength` Int16, `BrowserLanguage` FixedString(2), `BrowserCountry` FixedString(2), `SocialNetwork` String, `SocialAction` String, `HTTPError` UInt16, `SendTiming` Int32, `DNSTiming` Int32, `ConnectTiming` Int32, `ResponseStartTiming` Int32, `ResponseEndTiming` Int32, `FetchTiming` Int32, `RedirectTiming` Int32, `DOMInteractiveTiming` Int32, `DOMContentLoadedTiming` Int32, `DOMCompleteTiming` Int32, `LoadEventStartTiming` Int32, `LoadEventEndTiming` Int32, `NSToDOMContentLoadedTiming` Int32, `FirstPaintTiming` Int32, `RedirectCount` Int8, `SocialSourceNetworkID` UInt8, `SocialSourcePage` String, `ParamPrice` Int64, `ParamOrderID` String, `ParamCurrency` FixedString(3), `ParamCurrencyID` UInt16, `GoalsReached` Array(UInt32), `OpenstatServiceName` String, `OpenstatCampaignID` String, `OpenstatAdID` String, `OpenstatSourceID` String, `UTMSource` String, `UTMMedium` String, `UTMCampaign` String, `UTMContent` String, `UTMTerm` String, `FromTag` String, `HasGCLID` UInt8, `RefererHash` UInt64, `URLHash` UInt64, `CLID` UInt32, `YCLID` UInt64, `ShareService` String, `ShareURL` String, `ShareTitle` String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `IslandID` FixedString(16), `RequestNum` UInt32, `RequestTry` UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192, enable_mixed_granularity_parts=1, index_granularity_bytes=0; -- same with hits, but enabled mixed granularity and fixed_granularity
|
||||
|
||||
INSERT INTO mixed_granularity_strictly_non_adaptive_table SELECT * FROM test.hits LIMIT 10;
|
||||
|
||||
ALTER TABLE mixed_granularity_strictly_non_adaptive_table REPLACE PARTITION 201403 FROM mixed_granularity_table; -- { serverError 49 }
|
||||
|
||||
DROP TABLE IF EXISTS mixed_granularity_table;
|
||||
|
||||
DROP TABLE IF EXISTS mixed_granularity_strictly_non_adaptive_table;
|
Loading…
Reference in New Issue
Block a user