Maxwell-Guo commented on code in PR #2786:
URL: https://github.com/apache/cassandra/pull/2786#discussion_r1360595377
##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1441,14 +1446,62 @@ private static void
validateSSTableFormatFactories(Iterable<SSTableFormat.Factor
private static ImmutableMap<String, Supplier<SSTableFormat<?, ?>>>
validateAndMatchSSTableFormatOptions(Iterable<SSTableFormat.Factory> factories,
Map<String, Map<String, String>> options)
{
+ ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>>
providersBuilderLeft = ImmutableMap.builder();
+ ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>>
providersBuilderRight = ImmutableMap.builder();
ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>>
providersBuilder = ImmutableMap.builder();
if (options == null)
options = ImmutableMap.of();
+
+ // add default sstable format options for bti and big
for (SSTableFormat.Factory factory : factories)
{
- Map<String, String> formatOptions =
options.getOrDefault(factory.name(), ImmutableMap.of());
- providersBuilder.put(factory.name(), () ->
factory.getInstance(ImmutableMap.copyOf(formatOptions)));
+ providersBuilderLeft.put(factory.name(), () ->
factory.getInstance(SSTableFormatParams.DEFAULT_FORMAT_MAP.apply(factory.name())));
}
+
+ for (String key : options.keySet())
+ {
+ Map<String, String> formatOptions = Maps.newLinkedHashMap();
+ Map<String, String> map = options.get(key);
+ Iterator<String> iterator = map.keySet().iterator();
+ // as the map's value parsed by yaml may be numeric type, but
sstable format factory need string
+ // but what about change the type of sstable format factory to a
map that value is Object?
+ while (iterator.hasNext()){
+ String k = iterator.next();
+ String v = String.valueOf(map.get(k));
+ formatOptions.put(k, v);
+ }
+ // TODO add validation for yaml configuration number that must
meet all the sstable param options ?
Review Comment:
No, I don't think so, only if we can change the [factory
name](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java#L635)
with is big or bti.
as we only have big and bti format. But the kind of parameters for big and
bti may have more than two, According to my understanding to CASSANDRA-18534(If
my understanding is wrong, please help me point it out ,thanks) there may be
many custom-type that can be defined in yaml , for example , we may got
big-fast, big-small, big-slow, bti-one, bti-two, and so on. each of them with
different sstable parameters(I think it is ok to have same parameters wit
different type name), and users can define their sstable format through cql's
ddl , but the format should be defined in yaml before.
(1)My expectation might be there a map with many entries which key is the
format name (bit-one/ big-fast) value is the sstable parameters;
(2)but as factory name are big or bti , so the finally map number is only
two.
I originally wanted to make modifications as (1) described, but I think it
would be better to start after confirming with you, to confirm whether my
modifications meet your expectations.
And you can see that I have left a comment
[TODO](https://github.com/Maxwell-Guo/cassandra/blob/CASSANDRA-18534/src/java/org/apache/cassandra/db/memtable/Flushing.java#L107)
here , that is the same reason with (1). @blambov
##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1441,14 +1446,62 @@ private static void
validateSSTableFormatFactories(Iterable<SSTableFormat.Factor
private static ImmutableMap<String, Supplier<SSTableFormat<?, ?>>>
validateAndMatchSSTableFormatOptions(Iterable<SSTableFormat.Factory> factories,
Map<String, Map<String, String>> options)
{
+ ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>>
providersBuilderLeft = ImmutableMap.builder();
+ ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>>
providersBuilderRight = ImmutableMap.builder();
ImmutableMap.Builder<String, Supplier<SSTableFormat<?, ?>>>
providersBuilder = ImmutableMap.builder();
if (options == null)
options = ImmutableMap.of();
+
+ // add default sstable format options for bti and big
for (SSTableFormat.Factory factory : factories)
{
- Map<String, String> formatOptions =
options.getOrDefault(factory.name(), ImmutableMap.of());
- providersBuilder.put(factory.name(), () ->
factory.getInstance(ImmutableMap.copyOf(formatOptions)));
+ providersBuilderLeft.put(factory.name(), () ->
factory.getInstance(SSTableFormatParams.DEFAULT_FORMAT_MAP.apply(factory.name())));
}
+
+ for (String key : options.keySet())
+ {
+ Map<String, String> formatOptions = Maps.newLinkedHashMap();
+ Map<String, String> map = options.get(key);
+ Iterator<String> iterator = map.keySet().iterator();
+ // as the map's value parsed by yaml may be numeric type, but
sstable format factory need string
+ // but what about change the type of sstable format factory to a
map that value is Object?
+ while (iterator.hasNext()){
+ String k = iterator.next();
+ String v = String.valueOf(map.get(k));
+ formatOptions.put(k, v);
+ }
+ // TODO add validation for yaml configuration number that must
meet all the sstable param options ?
Review Comment:
No, I don't think so, only if we can change the [factory
name](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java#L635)
with is big or bti.
as we only have big and bti format. But the kind of parameters for big and
bti may have more than two, According to my understanding to CASSANDRA-18534(If
my understanding is wrong, please help me point it out ,thanks) there may be
many custom-type that can be defined in yaml , for example , we may got
big-fast, big-small, big-slow, bti-one, bti-two, and so on. each of them with
different sstable parameters(I think it is ok to have same parameters wit
different type name), and users can define their sstable format through cql's
ddl , but the format should be defined in yaml before.
(1)My expectation might be there a map with many entries which key is the
format name (bit-one/ big-fast) value is the sstable parameters;
(2)but as factory name are big or bti , so the finally map number is only
two.
I originally wanted to make modifications as (1) described, but I think it
would be better to start after confirming with you, to confirm whether my
modifications meet your expectations.
And you can see that I have left a comment
[TODO](https://github.com/Maxwell-Guo/cassandra/blob/CASSANDRA-18534/src/java/org/apache/cassandra/db/memtable/Flushing.java#L107)
here , that is the same reason with (1). @blambov
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]