Skip to content
GitLab
菜单
项目
群组
代码片段
/
帮助
帮助
支持
社区论坛
快捷键
?
提交反馈
登录/注册
切换导航
菜单
打开侧边栏
rui xia
frocksdb
提交
18203969
提交
18203969
编辑于
8月 27, 2021
作者:
rui xia
浏览文件
Extract key group to filter keys
上级
e515ea70
变更
5
Show whitespace changes
Inline
Side-by-side
java/rocksjni/flink_rescalingcompactionfilterjni.cc
浏览文件 @
18203969
...
...
@@ -67,32 +67,18 @@ jlong Java_org_rocksdb_FlinkRescalingCompactionFilter_createNewFlinkRescalingCom
* Signature: ?
*/
jboolean
Java_org_rocksdb_FlinkRescalingCompactionFilter_configureFlinkRescalingCompactionFilter
(
JNIEnv
*
env
,
jclass
/* jcls */
,
jlong
handle
,
jint
ji_rescale_round
,
jbyteArray
j_smallest_key
,
jint
j_smallest_key_len
,
jbyteArray
j_largest_key
,
jint
j_largest_key_len
)
{
JNIEnv
*
/* env */
,
jclass
/* jcls */
,
jlong
handle
,
jint
ji_rescale_round
,
jint
j_start_key_range
,
jint
j_end_key_range
,
jint
j_key_group_prefix_bytes
)
{
auto
rescale_round
=
static_cast
<
FlinkRescalingCompactionFilter
::
RescaleRound
>
(
ji_rescale_round
);
auto
config_holder
=
*
(
reinterpret_cast
<
std
::
shared_ptr
<
FlinkRescalingCompactionFilter
::
ConfigHolder
>*>
(
handle
));
jbyte
*
smallest_key
=
new
jbyte
[
j_smallest_key_len
];
env
->
GetByteArrayRegion
(
j_smallest_key
,
0
,
j_smallest_key_len
,
smallest_key
);
if
(
env
->
ExceptionCheck
())
{
// exception thrown: ArrayIndexOutOfBoundsException
delete
[]
smallest_key
;
return
false
;
}
jbyte
*
largest_key
=
new
jbyte
[
j_largest_key_len
];
env
->
GetByteArrayRegion
(
j_largest_key
,
0
,
j_largest_key_len
,
largest_key
);
if
(
env
->
ExceptionCheck
())
{
// exception thrown: ArrayIndexOutOfBoundsException
delete
[]
largest_key
;
return
false
;
}
auto
start_key_group
=
static_cast
<
uint32_t
>
(
j_start_key_range
);
auto
end_key_group
=
static_cast
<
uint32_t
>
(
j_end_key_range
);
auto
key_group_prefix_bytes
=
static_cast
<
uint32_t
>
(
j_key_group_prefix_bytes
);
ROCKSDB_NAMESPACE
::
Slice
smallest_key_slice
(
reinterpret_cast
<
char
*>
(
smallest_key
),
j_smallest_key_len
);
ROCKSDB_NAMESPACE
::
Slice
largest_key_slice
(
reinterpret_cast
<
char
*>
(
largest_key
),
j_largest_key_len
);
auto
config
=
new
FlinkRescalingCompactionFilter
::
Config
{
rescale_round
,
smallest_key_slice
,
largest_key_slice
};
auto
config
=
new
FlinkRescalingCompactionFilter
::
Config
{
rescale_round
,
start_key_group
,
end_key_group
,
key_group_prefix_bytes
};
return
static_cast
<
jboolean
>
(
config_holder
->
Configure
(
config
));
}
\ No newline at end of file
java/src/main/java/org/rocksdb/FlinkRescalingCompactionFilter.java
浏览文件 @
18203969
...
...
@@ -31,27 +31,28 @@ public class FlinkRescalingCompactionFilter extends AbstractCompactionFilter<Sli
private
native
static
long
createNewFlinkRescalingCompactionConfigHolder
();
private
native
static
void
disposeFlinkRescalingCompactionFilterConfigHolder
(
long
configHolderHandle
);
private
native
static
boolean
configureFlinkRescalingCompactionFilter
(
long
configHolderHandle
,
int
rescaleRound
,
byte
[]
smallestKey
,
int
smallestKeyLen
,
byte
[]
largestKey
,
int
largestKeyLen
);
int
rescaleRound
,
int
startKeyGroup
,
int
endKeyGroup
,
int
keyGroupPrefixBytes
);
public
static
class
Config
{
final
RescaleRound
rescaleRound
;
final
byte
[]
smallestKey
;
final
byte
[]
largestKey
;
final
int
startKeyGroup
;
final
int
endKeyGroup
;
final
int
keyGroupPrefixBytes
;
private
Config
(
RescaleRound
rescaleRound
,
byte
[]
smallestKey
,
byte
[]
largestKey
)
{
private
Config
(
RescaleRound
rescaleRound
,
int
startKeyGroup
,
int
endKeyGroup
,
int
keyGroupPrefixBytes
)
{
this
.
rescaleRound
=
rescaleRound
;
this
.
smallestKey
=
smallestKey
;
this
.
largestKey
=
largestKey
;
this
.
startKeyGroup
=
startKeyGroup
;
this
.
endKeyGroup
=
endKeyGroup
;
this
.
keyGroupPrefixBytes
=
keyGroupPrefixBytes
;
}
@SuppressWarnings
(
"WeakerAccess"
)
public
static
Config
createForZero
(
byte
[]
smallestKey
,
byte
[]
largestKey
)
{
return
new
Config
(
RescaleRound
.
Zero
,
s
mallestKey
,
largestKey
);
public
static
Config
createForZero
(
int
startKeyGroup
,
int
endKeyGroup
,
int
keyGroupPrefixBytes
)
{
return
new
Config
(
RescaleRound
.
Zero
,
s
tartKeyGroup
,
endKeyGroup
,
keyGroupPrefixBytes
);
}
public
static
Config
createForOne
(
byte
[]
smallestKey
,
byte
[]
largestKey
)
{
return
new
Config
(
RescaleRound
.
One
,
s
mallestKey
,
largestKey
);
public
static
Config
createForOne
(
int
startKeyGroup
,
int
endKeyGroup
,
int
keyGroupPrefixBytes
)
{
return
new
Config
(
RescaleRound
.
One
,
s
tartKeyGroup
,
endKeyGroup
,
keyGroupPrefixBytes
);
}
}
...
...
@@ -105,8 +106,7 @@ public class FlinkRescalingCompactionFilter extends AbstractCompactionFilter<Sli
public
void
configure
(
Config
config
)
{
boolean
already_configured
=
!
configureFlinkRescalingCompactionFilter
(
configHolder
.
nativeHandle_
,
config
.
rescaleRound
.
ordinal
(),
config
.
smallestKey
,
config
.
smallestKey
.
length
,
config
.
largestKey
,
config
.
largestKey
.
length
);
config
.
startKeyGroup
,
config
.
endKeyGroup
,
config
.
keyGroupPrefixBytes
);
if
(
already_configured
)
{
throw
new
IllegalStateException
(
"Compaction filter is already configured"
);
}
...
...
utilities/flink/flink_rescaling_compaction_filter.cc
浏览文件 @
18203969
...
...
@@ -8,8 +8,15 @@
namespace
ROCKSDB_NAMESPACE
{
namespace
flink
{
int
compare
(
const
Slice
&
a
,
const
Slice
&
b
)
{
return
a
.
compare
(
b
);
uint32_t
get_key_group
(
const
Slice
&
key
,
int
key_group_prefix_bytes
)
{
// de-serialize key-group in the prefix of the key
uint32_t
key_group
=
0
;
for
(
int
i
=
0
;
i
<
key_group_prefix_bytes
;
++
i
)
{
key_group
<<=
8
;
key_group
|=
(
key
[
i
]
&
0xFF
);
}
return
key_group
;
}
FlinkRescalingCompactionFilter
::
ConfigHolder
::
ConfigHolder
()
...
...
@@ -57,37 +64,23 @@ const char* FlinkRescalingCompactionFilter::Name() const {
CompactionFilter
::
Decision
FlinkRescalingCompactionFilter
::
FilterV2
(
int
/* level */
,
const
Slice
&
key
,
ValueType
/* value_type */
,
const
Slice
&
existing_value
,
std
::
string
*
new_value
,
const
Slice
&
/*
existing_value
*/
,
std
::
string
*
/*
new_value
*/
,
std
::
string
*
/*skip_until*/
)
const
{
InitConfigIfNotYet
();
const
char
key_rescale_round
=
existing_value
.
data
()[
0
]
;
const
uint32_t
key_group
=
get_key_group
(
key
,
config_cached_
->
key_group_prefix_bytes_
)
;
Debug
(
logger_
.
get
(),
"Call FlinkRescalingCompactionFilter::FilterV2 - Key %s, rescale byte %d, "
"Rescaling round: %d, smallest key: %s, largest key: %s"
,
key
.
ToString
().
c_str
(),
config_cached_
->
rescale_round_
,
key_rescale_round
,
config_cached_
->
smallest_key_
.
ToString
().
c_str
(),
config_cached_
->
largest_key_
.
ToString
().
c_str
());
const
RescaleRound
rescale_round
=
config_cached_
->
rescale_round_
;
if
(
key_rescale_round
==
rescale_round
)
{
return
Decision
::
kKeep
;
}
"Call FlinkRescalingCompactionFilter::FilterV2 - Key %s (key group %d), "
"smallest key group: %d, largest key group: %d"
,
key
.
ToString
().
c_str
(),
key_group
,
config_cached_
->
start_key_group_
,
config_cached_
->
end_key_group_
);
// if key is in range [smallest, largest]
if
(
compare
(
key
,
config_cached_
->
smallest_key_
)
>=
0
&&
compare
(
key
,
config_cached_
->
largest_key_
)
<=
0
)
{
new_value
->
clear
();
new_value
->
assign
(
existing_value
.
data
(),
existing_value
.
size
());
(
*
new_value
)[
0
]
=
(
char
)
rescale_round
;
Logger
*
logger
=
logger_
.
get
();
if
(
logger
&&
logger
->
GetInfoLogLevel
()
<=
InfoLogLevel
::
DEBUG_LEVEL
)
{
Debug
(
logger
,
"New value: %s"
,
new_value
->
c_str
());
}
return
Decision
::
kChangeValue
;
if
(
key_group
>=
config_cached_
->
start_key_group_
&&
key_group
<=
config_cached_
->
end_key_group_
)
{
return
Decision
::
kKeep
;
}
else
{
return
Decision
::
kRemove
;
}
...
...
utilities/flink/flink_rescaling_compaction_filter.h
浏览文件 @
18203969
...
...
@@ -31,10 +31,11 @@ class FlinkRescalingCompactionFilter : public CompactionFilter {
One
};
struct
Config
{
struct
Config
{
RescaleRound
rescale_round_
;
Slice
smallest_key_
;
Slice
largest_key_
;
uint32_t
start_key_group_
;
uint32_t
end_key_group_
;
uint32_t
key_group_prefix_bytes_
;
};
class
ConfigHolder
{
...
...
utilities/flink/flink_rescaling_compaction_filter_test.cc
浏览文件 @
18203969
...
...
@@ -41,8 +41,8 @@ std::string new_list = ""; // NOLINT
std
::
string
stub
=
""
;
// NOLINT
FlinkRescalingCompactionFilter
::
RescaleRound
rescale_round
;
Slice
largest_key
;
Slice
smallest_key
;
uint32_t
end_key_group
;
uint32_t
start_key_group
;
CompactionFilter
::
ValueType
value_type
;
FlinkRescalingCompactionFilter
*
filter
;
// NOLINT
...
...
@@ -53,12 +53,12 @@ CompactionFilter::Decision decide() {
void
Init
(
FlinkRescalingCompactionFilter
::
RescaleRound
rround
,
FlinkRescalingCompactionFilter
::
RescaleRound
vround
,
const
Slice
&
skey
,
const
Slice
&
lkey
,
const
Slice
&
vkey
=
Slice
(
"key"
))
{
const
uint32_t
skey_group
,
const
uint32_t
ekey_group
,
const
Slice
&
vkey
=
Slice
(
std
::
string
(
new
char
[
5
]{
10
,
'0'
,
'0'
,
'0'
,
0
},
5
)
))
{
rescale_round
=
rround
;
s
mallest_key
=
skey
;
largest_key
=
l
key
;
s
tart_key_group
=
skey
_group
;
end_key_group
=
e
key
_group
;
value_type
=
CompactionFilter
::
ValueType
::
kValue
;
data
[
0
]
=
vround
;
key
=
vkey
;
...
...
@@ -67,7 +67,7 @@ void Init(
auto
logger
=
std
::
make_shared
<
ConsoleLogger
>
();
filter
=
new
FlinkRescalingCompactionFilter
(
config_holder
,
logger
);
auto
config
=
new
FlinkRescalingCompactionFilter
::
Config
{
rescale_round
,
s
mallest_key
,
largest_key
};
auto
config
=
new
FlinkRescalingCompactionFilter
::
Config
{
rescale_round
,
s
tart_key_group
,
end_key_group
,
1
};
EXPECT_TRUE
(
config_holder
->
Configure
(
config
));
EXPECT_FALSE
(
config_holder
->
Configure
(
config
));
}
...
...
@@ -83,7 +83,7 @@ TEST(FlinkStateRescaleTest, CheckRescaleRoundEnumOrder) {
TEST
(
FlinkStateRescaleTest
,
CurrentRound
)
{
Init
(
FlinkRescalingCompactionFilter
::
RescaleRound
::
One
,
FlinkRescalingCompactionFilter
::
RescaleRound
::
One
,
Slice
(
"000"
),
Slice
(
"111"
)
);
0
,
127
);
EXPECT_EQ
(
decide
(),
KKEEP
);
Deinit
();
}
...
...
@@ -91,7 +91,7 @@ TEST(FlinkStateRescaleTest, CurrentRound) {
TEST
(
FlinkStateRescaleTest
,
CurrentRound2
)
{
Init
(
FlinkRescalingCompactionFilter
::
RescaleRound
::
Zero
,
FlinkRescalingCompactionFilter
::
RescaleRound
::
Zero
,
Slice
(
"000"
),
Slice
(
"111"
)
);
0
,
127
);
EXPECT_EQ
(
decide
(),
KKEEP
);
Deinit
();
}
...
...
@@ -99,29 +99,27 @@ TEST(FlinkStateRescaleTest, CurrentRound2) {
TEST
(
FlinkStateRescaleTest
,
NotCurrentRoundInRange
)
{
Init
(
FlinkRescalingCompactionFilter
::
RescaleRound
::
Zero
,
FlinkRescalingCompactionFilter
::
RescaleRound
::
One
,
Slice
(
"000"
),
Slice
(
"111"
),
Slice
(
"000"
));
EXPECT_EQ
(
decide
(),
KCHANGE
);
EXPECT_EQ
(
new_list
.
data
()[
0
],
FlinkRescalingCompactionFilter
::
RescaleRound
::
Zero
);
0
,
127
,
Slice
(
std
::
string
(
new
char
[
5
]{
0
,
'0'
,
'0'
,
'0'
,
0
},
5
)));
EXPECT_EQ
(
decide
(),
KKEEP
);
Deinit
();
Init
(
FlinkRescalingCompactionFilter
::
RescaleRound
::
Zero
,
FlinkRescalingCompactionFilter
::
RescaleRound
::
One
,
Slice
(
"000"
),
Slice
(
"111"
),
Slice
(
"111"
));
EXPECT_EQ
(
decide
(),
KCHANGE
);
EXPECT_EQ
(
new_list
.
data
()[
0
],
FlinkRescalingCompactionFilter
::
RescaleRound
::
Zero
);
0
,
127
,
Slice
(
std
::
string
(
new
char
[
5
]{
127
,
'0'
,
'0'
,
'0'
,
0
},
5
)));
EXPECT_EQ
(
decide
(),
KKEEP
);
Deinit
();
}
TEST
(
FlinkStateRescaleTest
,
NotCurrentRoundNotInRange
)
{
Init
(
FlinkRescalingCompactionFilter
::
RescaleRound
::
Zero
,
FlinkRescalingCompactionFilter
::
RescaleRound
::
One
,
Slice
(
"100"
),
Slice
(
"111"
),
Slice
(
"000"
));
0
,
0
,
Slice
(
std
::
string
(
new
char
[
5
]{
1
,
'0'
,
'0'
,
'0'
,
0
},
5
)
));
EXPECT_EQ
(
decide
(),
KREMOVE
);
Deinit
();
Init
(
FlinkRescalingCompactionFilter
::
RescaleRound
::
Zero
,
FlinkRescalingCompactionFilter
::
RescaleRound
::
One
,
Slice
(
"000"
),
Slice
(
"110"
),
Slice
(
"111"
));
0
,
126
,
Slice
(
std
::
string
(
new
char
[
5
]{
127
,
'0'
,
'0'
,
'0'
,
0
},
5
)
));
EXPECT_EQ
(
decide
(),
KREMOVE
);
Deinit
();
}
...
...
编辑
预览
支持
Markdown
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录