blambov commented on code in PR #2064:
URL: https://github.com/apache/cassandra/pull/2064#discussion_r1093293126
##########
src/java/org/apache/cassandra/io/sstable/Component.java:
##########
@@ -41,55 +39,76 @@
{
public static final char separator = '-';
- final static ImmutableSet<Type> TYPES = Type.all;
+ @SuppressWarnings("rawtypes")
+ public static Set<Component> getSingletonsFor(Class<? extends
SSTableFormat> format)
+ {
+ return singletons.stream().filter(c ->
c.type.formatClass.isAssignableFrom(format)).collect(Collectors.toSet());
+ }
/**
* WARNING: Be careful while changing the names or string representation
of the enum
* members. Streaming code depends on the names during streaming (Ref:
CASSANDRA-14556).
*/
- public static class Type
+ public final static class Type
{
- private final static List<Type> singletonsCollector = new
ArrayList<>(11);
+ private final static CopyOnWriteArrayList<Type> typesCollector = new
CopyOnWriteArrayList<>();
// the base data for an sstable: the remaining components can be
regenerated
// based on the data component
- public static final Type DATA = new Type("DATA", "Data.db", name ->
Component.DATA);
+ public static final Type DATA = new Type("DATA", "Data.db", name ->
Component.DATA, true);
// file to hold information about uncompressed data length, chunk
offsets etc.
- public static final Type COMPRESSION_INFO = new
Type("COMPRESSION_INFO", "CompressionInfo.db", name ->
Component.COMPRESSION_INFO);
+ public static final Type COMPRESSION_INFO = new
Type("COMPRESSION_INFO", "CompressionInfo.db", name ->
Component.COMPRESSION_INFO, true);
// statistical metadata about the content of the sstable
- public static final Type STATS = new Type("STATS", "Statistics.db",
name -> Component.STATS);
+ public static final Type STATS = new Type("STATS", "Statistics.db",
name -> Component.STATS, true);
// serialized bloom filter for the row keys in the sstable
- public static final Type FILTER = new Type("FILTER", "Filter.db", name
-> Component.FILTER);
+ public static final Type FILTER = new Type("FILTER", "Filter.db", name
-> Component.FILTER, true);
// holds CRC32 checksum of the data file
- public static final Type DIGEST = new Type("DIGEST","Digest.crc32",
name -> Component.DIGEST);
- // holds the CRC32 for chunks in an a uncompressed file.
- public static final Type CRC = new Type("CRC","CRC.db", name ->
Component.CRC);
+ public static final Type DIGEST = new Type("DIGEST","Digest.crc32",
name -> Component.DIGEST, true);
+ // holds the CRC32 for chunks in an uncompressed file.
+ public static final Type CRC = new Type("CRC","CRC.db", name ->
Component.CRC, true);
// table of contents, stores the list of all components for the sstable
- public static final Type TOC = new Type("TOC","TOC.txt", name ->
Component.TOC);
+ public static final Type TOC = new Type("TOC","TOC.txt", name ->
Component.TOC, true);
// built-in secondary index (may be multiple per sstable)
- public static final Type SECONDARY_INDEX = new
Type("SECONDARY_INDEX","SI_.*.db", name -> new Component(Type.SECONDARY_INDEX,
name));
+ public static final Type SECONDARY_INDEX = new
Type("SECONDARY_INDEX","SI_.*.db", name -> new Component(Type.SECONDARY_INDEX,
name), false);
// custom component, used by e.g. custom compaction strategy
- public static final Type CUSTOM = new Type("CUSTOM",null, name -> new
Component(Type.CUSTOM, name));
+ public static final Type CUSTOM = new Type("CUSTOM",null, name -> new
Component(Type.CUSTOM, name), false);
- public static final ImmutableSet<Type> all =
ImmutableSet.copyOf(singletonsCollector);
+ public static final List<Type> all =
Collections.unmodifiableList(typesCollector);
public final int id;
public final String name;
public final String repr;
- private Function<String, Component> componentFactory;
+ private final Function<String, Component> componentFactory;
+ public final boolean isSingleton;
+
+ @SuppressWarnings("rawtypes")
+ public final Class<? extends SSTableFormat> formatClass;
+
+ private Type(String name, String repr, Function<String, Component>
componentFactory, boolean isSingleton)
+ {
+ this(name, repr, componentFactory, isSingleton,
SSTableFormat.class);
+ }
- protected Type(String name, String repr, Function<String, Component>
componentFactory)
+ @SuppressWarnings("rawtypes")
+ public Type(String name, String repr, Function<String, Component>
componentFactory, boolean isSingleton, Class<? extends SSTableFormat>
formatClass)
{
this.name = name;
this.repr = repr;
- this.id = singletonsCollector.size();
+ this.id = typesCollector.size();
this.componentFactory = componentFactory;
- singletonsCollector.add(this);
+ this.isSingleton = isSingleton;
+ this.formatClass = formatClass;
+ synchronized (typesCollector)
+ {
+ if (typesCollector.stream().anyMatch(t ->
Objects.equal(t.name, name) && t.formatClass == formatClass ||
t.formatClass.isAssignableFrom(formatClass) ||
formatClass.isAssignableFrom(t.formatClass)))
Review Comment:
I mean that a subclass of `Component.Type` will always fire
`t.formatClass.isAssignableFrom(formatClass)` (e.g. for `t = TOC`) and thus the
whole condition without checking `Objects.equal(t.name, name)`.
--
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]