You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2021/12/01 13:09:37 UTC

[GitHub] [cassandra] smiklosovic opened a new pull request #1351: CASSANDRA-17180 - implement heartbeat service

smiklosovic opened a new pull request #1351:
URL: https://github.com/apache/cassandra/pull/1351


   WIP


-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a change in pull request #1351: CASSANDRA-17180

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on a change in pull request #1351:
URL: https://github.com/apache/cassandra/pull/1351#discussion_r818068501



##########
File path: src/java/org/apache/cassandra/service/GcGraceSecondsOnStartupCheck.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.Collectors;
+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.Clock;
+import org.apache.cassandra.utils.Pair;
+
+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_MACHINE_STATE;
+
+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<>();
+        long currentTimeMillis = Clock.Global.currentTimeMillis();
+
+        Set<String> excludedKeyspaces = getExcludedKeyspaces(config);
+        Set<Pair<String, String>> excludedTables = getExcludedTables(config);
+
+        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)

Review comment:
       @pauloricardomg could you please double check this logic, in particular?




-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a change in pull request #1351: CASSANDRA-17180

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on a change in pull request #1351:
URL: https://github.com/apache/cassandra/pull/1351#discussion_r822950621



##########
File path: src/java/org/apache/cassandra/service/HeartbeatService.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.ScheduledExecutorPlus;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.StartupChecksOptions;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.ExecutorUtils;
+
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
+import static org.apache.cassandra.service.GcGraceSecondsOnStartupCheck.getHeartbeatFile;
+import static org.apache.cassandra.service.StartupChecks.StartupCheckType.gc_grace_period;
+
+public class HeartbeatService
+{
+    private static final Logger logger = LoggerFactory.getLogger(HeartbeatService.class);
+
+    public static final HeartbeatService instance = new HeartbeatService();
+    private ScheduledExecutorPlus executor;
+    private long delay = 1;
+    private TimeUnit delayTimeUnit = TimeUnit.MINUTES;
+
+    private boolean started = false;
+    private Runnable heartbeat;
+
+    @VisibleForTesting
+    HeartbeatService()
+    {
+    }
+
+    @VisibleForTesting
+    void setHeartbeat(final Runnable heartbeat)
+    {
+        this.heartbeat = heartbeat;
+    }
+
+    @VisibleForTesting
+    void setDelay(long delay, TimeUnit delayTimeUnit)
+    {
+        if (delay > 0 && delayTimeUnit != null)
+        {
+            this.delay = delay;
+            this.delayTimeUnit = delayTimeUnit;
+        }
+    }
+
+    @VisibleForTesting
+    void setExecutor(final ScheduledExecutorPlus executor)
+    {
+        if (started)
+            throw new IllegalStateException("Can not set executor when service is started. Stop it first.");
+
+        this.executor = executor;
+    }
+
+    @VisibleForTesting
+    StartupChecksOptions getStartupChecksOptions()
+    {
+        return DatabaseDescriptor.getStartupChecksOptions();
+    }
+
+    public synchronized void start()
+    {
+        if (started)
+            return;
+
+        if (!getStartupChecksOptions().isEnabled(gc_grace_period))
+        {
+            logger.debug("Heartbeat service is disabled.");
+            return;
+        }
+
+        if (heartbeat == null)
+            heartbeat = new Heartbeat(getHeartbeatFile(getStartupChecksOptions().getConfig(gc_grace_period)));
+
+        if (executor == null)
+            executor = executorFactory().scheduled(false, "HeartbeatService");
+
+        if (executor.isShutdown())
+            throw new IllegalStateException("Executor to run heartbeats on is shut down!");
+
+        executor.scheduleWithFixedDelay(heartbeat, 0, delay, delayTimeUnit);
+
+        started = true;
+    }
+
+    public synchronized void stop() throws InterruptedException, TimeoutException
+    {
+        if (executor != null)
+            ExecutorUtils.shutdownNowAndWait(1L, TimeUnit.MINUTES, executor);
+
+        executor = null;
+        started = false;
+    }
+
+    private static class Heartbeat implements Runnable
+    {
+        private final File heartbeatFile;
+
+        public Heartbeat(File heartbeatFile)
+        {
+            this.heartbeatFile = heartbeatFile;
+        }
+
+        @Override
+        public void run()
+        {
+            FileUtils.write(heartbeatFile, Long.toString(Clock.Global.currentTimeMillis()));

Review comment:
       Should be timestamp written in the file rather human readable for easier manual verification / inspection? It is quite hard to grasp what was the last time the heartbeat file was written to when one sees just a number only.




-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a change in pull request #1351: CASSANDRA-17180

Posted by GitBox <gi...@apache.org>.
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a change in pull request #1351: CASSANDRA-17180

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on a change in pull request #1351:
URL: https://github.com/apache/cassandra/pull/1351#discussion_r822950621



##########
File path: src/java/org/apache/cassandra/service/HeartbeatService.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.ScheduledExecutorPlus;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.StartupChecksOptions;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.ExecutorUtils;
+
+import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory;
+import static org.apache.cassandra.service.GcGraceSecondsOnStartupCheck.getHeartbeatFile;
+import static org.apache.cassandra.service.StartupChecks.StartupCheckType.gc_grace_period;
+
+public class HeartbeatService
+{
+    private static final Logger logger = LoggerFactory.getLogger(HeartbeatService.class);
+
+    public static final HeartbeatService instance = new HeartbeatService();
+    private ScheduledExecutorPlus executor;
+    private long delay = 1;
+    private TimeUnit delayTimeUnit = TimeUnit.MINUTES;
+
+    private boolean started = false;
+    private Runnable heartbeat;
+
+    @VisibleForTesting
+    HeartbeatService()
+    {
+    }
+
+    @VisibleForTesting
+    void setHeartbeat(final Runnable heartbeat)
+    {
+        this.heartbeat = heartbeat;
+    }
+
+    @VisibleForTesting
+    void setDelay(long delay, TimeUnit delayTimeUnit)
+    {
+        if (delay > 0 && delayTimeUnit != null)
+        {
+            this.delay = delay;
+            this.delayTimeUnit = delayTimeUnit;
+        }
+    }
+
+    @VisibleForTesting
+    void setExecutor(final ScheduledExecutorPlus executor)
+    {
+        if (started)
+            throw new IllegalStateException("Can not set executor when service is started. Stop it first.");
+
+        this.executor = executor;
+    }
+
+    @VisibleForTesting
+    StartupChecksOptions getStartupChecksOptions()
+    {
+        return DatabaseDescriptor.getStartupChecksOptions();
+    }
+
+    public synchronized void start()
+    {
+        if (started)
+            return;
+
+        if (!getStartupChecksOptions().isEnabled(gc_grace_period))
+        {
+            logger.debug("Heartbeat service is disabled.");
+            return;
+        }
+
+        if (heartbeat == null)
+            heartbeat = new Heartbeat(getHeartbeatFile(getStartupChecksOptions().getConfig(gc_grace_period)));
+
+        if (executor == null)
+            executor = executorFactory().scheduled(false, "HeartbeatService");
+
+        if (executor.isShutdown())
+            throw new IllegalStateException("Executor to run heartbeats on is shut down!");
+
+        executor.scheduleWithFixedDelay(heartbeat, 0, delay, delayTimeUnit);
+
+        started = true;
+    }
+
+    public synchronized void stop() throws InterruptedException, TimeoutException
+    {
+        if (executor != null)
+            ExecutorUtils.shutdownNowAndWait(1L, TimeUnit.MINUTES, executor);
+
+        executor = null;
+        started = false;
+    }
+
+    private static class Heartbeat implements Runnable
+    {
+        private final File heartbeatFile;
+
+        public Heartbeat(File heartbeatFile)
+        {
+            this.heartbeatFile = heartbeatFile;
+        }
+
+        @Override
+        public void run()
+        {
+            FileUtils.write(heartbeatFile, Long.toString(Clock.Global.currentTimeMillis()));

Review comment:
       Should be timestamp written in the file rather human readable for easier manual verification / inspection? It is quite hard to grasp what was the last time the heartbeat file was written to when one see just a number only.




-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org