smiklosovic commented on a change in pull request #1351:
URL: https://github.com/apache/cassandra/pull/1351#discussion_r822947601



##########
File path: 
src/java/org/apache/cassandra/service/GcGraceSecondsOnStartupCheck.java
##########
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service;
+
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.StreamSupport;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.StartupChecksOptions;
+import org.apache.cassandra.exceptions.StartupException;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.utils.Pair;
+
+import static java.util.stream.Collectors.joining;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toSet;
+import static 
org.apache.cassandra.config.CassandraRelevantProperties.LINE_SEPARATOR;
+import static 
org.apache.cassandra.exceptions.StartupException.ERR_WRONG_DISK_STATE;
+import static 
org.apache.cassandra.exceptions.StartupException.ERR_WRONG_MACHINE_STATE;
+import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
+
+public class GcGraceSecondsOnStartupCheck implements StartupCheck
+{
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(GcGraceSecondsOnStartupCheck.class);
+
+    public static final String HEARTBEAT_FILE_CONFIG_PROPERTY = 
"heartbeat_file";
+    public static final String EXCLUDED_KEYSPACES_CONFIG_PROPERTY = 
"excluded_keyspaces";
+    public static final String EXCLUDED_TABLES_CONFIG_PROPERTY = 
"excluded_tables";
+
+    public static final String DEFAULT_HEARTBEAT_FILE = ".cassandra-heartbeat";
+
+    @Override
+    public StartupChecks.StartupCheckType getStartupCheckType()
+    {
+        return StartupChecks.StartupCheckType.gc_grace_period;
+    }
+
+    static File getHeartbeatFile(Map<String, Object> config)
+    {
+        String heartbeatFileConfigValue = (String) 
config.get(HEARTBEAT_FILE_CONFIG_PROPERTY);
+
+        return heartbeatFileConfigValue == null
+               ? new File(DEFAULT_HEARTBEAT_FILE)
+               : new File(heartbeatFileConfigValue);
+    }
+
+    @VisibleForTesting
+    public Set<String> getExcludedKeyspaces(Map<String, Object> config)
+    {
+        String excludedKeyspacesConfigValue = (String) 
config.get(EXCLUDED_KEYSPACES_CONFIG_PROPERTY);
+
+        if (excludedKeyspacesConfigValue == null)
+            return Collections.emptySet();
+        else
+            return 
Arrays.stream(excludedKeyspacesConfigValue.trim().split(","))
+                         .map(String::trim)
+                         .collect(toSet());
+    }
+
+    @VisibleForTesting
+    public Set<Pair<String, String>> getExcludedTables(Map<String, Object> 
config)
+    {
+        String excludedKeyspacesConfigValue = (String) 
config.get(EXCLUDED_TABLES_CONFIG_PROPERTY);
+
+        if (excludedKeyspacesConfigValue == null)
+            return Collections.emptySet();
+
+        Set<Pair<String, String>> pairs = new HashSet<>();
+
+        for (String keyspaceTable : 
excludedKeyspacesConfigValue.trim().split(","))
+        {
+            String[] pair = keyspaceTable.trim().split("\\.");
+            if (pair.length != 2)
+                continue;
+
+            pairs.add(Pair.create(pair[0].trim(), pair[1].trim()));
+        }
+
+        return pairs;
+    }
+
+    @VisibleForTesting
+    List<String> getKeyspaces()
+    {
+        return Schema.instance.getUserKeyspaces();
+    }
+
+    @VisibleForTesting
+    List<Pair<String, Integer>> getTablesGcGraceSecondsPairs(String 
userKeyspace)
+    {
+        return 
StreamSupport.stream(Schema.instance.getTablesAndViews(userKeyspace).spliterator(),
 false)
+                            .map(tableMetadata -> 
Pair.create(tableMetadata.name, tableMetadata.params.gcGraceSeconds))
+                            .collect(toList());
+    }
+
+    @Override
+    public void execute(StartupChecksOptions options) throws StartupException
+    {
+        if (options.isDisabled(getStartupCheckType()))
+            return;
+
+        Map<String, Object> config = options.getConfig(getStartupCheckType());
+        File heartbeatFile = getHeartbeatFile(config);
+
+        if (!heartbeatFile.exists())
+        {
+            LOGGER.debug("Heartbeat file {} not found! Skipping heartbeat 
startup check.",
+                         heartbeatFile.absolutePath());
+            return;
+        }
+
+        // we expect heartbeat value to be on the first line
+        Optional<Long> heartbeatOptional = parseHeartbeatFile(heartbeatFile);
+        if (!heartbeatOptional.isPresent())
+            return;
+
+        long heartbeat = heartbeatOptional.get();
+
+        List<Pair<String, String>> violations = new ArrayList<>();
+
+        Set<String> excludedKeyspaces = getExcludedKeyspaces(config);
+        Set<Pair<String, String>> excludedTables = getExcludedTables(config);
+
+        long currentTimeMillis = currentTimeMillis();
+
+        for (String keyspace : getKeyspaces())
+        {
+            if (excludedKeyspaces.contains(keyspace))
+                continue;
+
+            for (Pair<String, Integer> userTable : 
getTablesGcGraceSecondsPairs(keyspace))
+            {
+                if (excludedTables.contains(Pair.create(keyspace, 
userTable.left)))
+                    continue;
+
+                long gcGraceMillis = ((long) userTable.right) * 1000;
+                if (heartbeat + gcGraceMillis < currentTimeMillis)
+                    violations.add(Pair.create(keyspace, userTable.left));
+            }
+        }
+
+        if (!violations.isEmpty())
+        {
+            String invalidTables = violations.stream()
+                                             .map(p -> String.format("%s.%s", 
p.left, p.right))
+                                             .collect(joining(","));
+
+            String exceptionMessage = String.format("There are tables for 
which gcGraceSeconds is older " +
+                                                    "then the lastly known 
time Cassandra node was up based " +
+                                                    "on its heartbeat. 
Cassandra node will not start " +

Review comment:
       TODO: Write heartbeat timestamp here, explicitly.




-- 
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]

Reply via email to