/*
* 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.io.File;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.net.InetAddress;
import java.util.Arrays;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import javax.management.MBeanServer;
import javax.management.ObjectName;
import javax.management.StandardMBean;
import com.addthis.metrics.reporter.config.ReporterConfig;
import com.google.common.collect.Iterables;
import com.google.common.collect.SetMultimap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.commitlog.CommitLog;
import org.apache.cassandra.db.compaction.CompactionManager;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.FSError;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.metrics.StorageMetrics;
import org.apache.cassandra.thrift.ThriftServer;
import org.apache.cassandra.tracing.Tracing;
import org.apache.cassandra.utils.CLibrary;
import org.apache.cassandra.utils.Mx4jTool;
import org.apache.cassandra.utils.Pair;
import org.apache.zookeeper.server.ZooKeeperServer;
import org.apache.zookeeper.server.quorum.QuorumPeerMain;
import org.apache.zookeeper.server.quorum.ZooEmbedded;
import org.apache.zookeeper.server.quorum.ZooQuorum;
import org.apache.zookeeper.server.quorum.ZooStandalone;
/**
* The <code>CassandraDaemon</code> is an abstraction for a Cassandra daemon
* service, which defines not only a way to activate and deactivate it, but also
* hooks into its lifecycle methods (see {@link #setup()}, {@link #start()},
* {@link #stop()} and {@link #setup()}).
*/
public class CassandraDaemon
{
public static final String MBEAN_NAME = "org.apache.cassandra.db:type=NativeAccess";
private static final Logger logger = LoggerFactory.getLogger(CassandraDaemon.class);
public static ZooEmbedded ZooServer = null;
private static String [] ConfDir;
// Have a dedicated thread to call exit to avoid deadlock in the case where the thread that wants to invoke exit
// belongs to an executor that our shutdown hook wants to wait to exit gracefully. See CASSANDRA-5273.
private static final Thread exitThread = new Thread(new Runnable()
{
public void run()
{
System.exit(100);
}
}, "Exit invoker");
//Timer for compaction
public static long compactionTimer;
//pgaref Zookeeper Thread
private static final Thread ZookeeperThread = new Thread(new Runnable()
{
public void run()
{
logger.info("Starting ZK-Thread..................");
ZooServer = new ZooQuorum();
ZooServer.init();
}
}, "Exit invoker");
private static final CassandraDaemon instance = new CassandraDaemon();
public Server thriftServer;
public Server nativeServer;
/**
* This is a hook for concrete daemons to initialize themselves suitably.
*
* Subclasses should override this to finish the job (listening on ports, etc.)
*
* @throws IOException
*/
protected void setup()
{
//pgaref Timer
compactionTimer = System.currentTimeMillis();
// log warnings for different kinds of sub-optimal JVMs. tldr use 64-bit Oracle >= 1.6u32
if (!System.getProperty("os.arch").contains("64"))
logger.info("32bit JVM detected. It is recommended to run Cassandra on a 64bit JVM for better performance.");
String javaVersion = System.getProperty("java.version");
String javaVmName = System.getProperty("java.vm.name");
logger.info("JVM vendor/version: {}/{}", javaVmName, javaVersion);
if (javaVmName.contains("OpenJDK"))
{
// There is essentially no QA done on OpenJDK builds, and
// clusters running OpenJDK have seen many heap and load issues.
logger.warn("OpenJDK is not recommended. Please upgrade to the newest Oracle Java release");
}
else if (!javaVmName.contains("HotSpot"))
{
logger.warn("Non-Oracle JVM detected. Some features, such as immediate unmap of compacted SSTables, may not work as intended");
}
/* else
{
String[] java_version = javaVersion.split("_");
String java_major = java_version[0];
int java_minor;
try
{
java_minor = (java_version.length > 1) ? Integer.parseInt(java_version[1]) : 0;
}
catch (NumberFormatException e)
{
// have only seen this with java7 so far but no doubt there are other ways to break this
logger.info("Unable to parse java version {}", Arrays.toString(java_version));
java_minor = 32;
}
}
*/
logger.info("Heap size: {}/{}", Runtime.getRuntime().totalMemory(), Runtime.getRuntime().maxMemory());
logger.info("Classpath: {}", System.getProperty("java.class.path"));
CLibrary.tryMlockall();
Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler()
{
public void uncaughtException(Thread t, Throwable e)
{
StorageMetrics.exceptions.inc();
logger.error("Exception in thread " + t, e);
Tracing.trace("Exception in thread " + t, e);
for (Throwable e2 = e; e2 != null; e2 = e2.getCause())
{
// some code, like FileChannel.map, will wrap an OutOfMemoryError in another exception
if (e2 instanceof OutOfMemoryError)
exitThread.start();
if (e2 instanceof FSError)
{
if (e2 != e) // make sure FSError gets logged exactly once.
logger.error("Exception in thread " + t, e2);
FileUtils.handleFSError((FSError) e2);
}
}
}
});
// check all directories(data, commitlog, saved cache) for existence and permission
Iterable<String> dirs = Iterables.concat(Arrays.asList(DatabaseDescriptor.getAllDataFileLocations()),
Arrays.asList(DatabaseDescriptor.getCommitLogLocation(),
DatabaseDescriptor.getSavedCachesLocation()));
for (String dataDir : dirs)
{
logger.debug("Checking directory {}", dataDir);
File dir = new File(dataDir);
if (dir.exists())
assert dir.isDirectory() && dir.canRead() && dir.canWrite() && dir.canExecute()
: String.format("Directory %s is not accessible.", dataDir);
}
if (CacheService.instance == null) // should never happen
throw new RuntimeException("Failed to initialize Cache Service.");
// check the system keyspace to keep user from shooting self in foot by changing partitioner, cluster name, etc.
// we do a one-off scrub of the system keyspace first; we can't load the list of the rest of the keyspaces,
// until system keyspace is opened.
for (CFMetaData cfm : Schema.instance.getKeyspaceMetaData(Keyspace.SYSTEM_KS).values())
ColumnFamilyStore.scrubDataDirectories(Keyspace.SYSTEM_KS, cfm.cfName);
try
{
SystemKeyspace.checkHealth();
}
catch (ConfigurationException e)
{
logger.error("Fatal exception during initialization", e);
System.exit(100);
}
// load keyspace descriptions.
DatabaseDescriptor.loadSchemas();
// clean up debris in the rest of the keyspaces
for (String keyspaceName : Schema.instance.getKeyspaces())
{
for (CFMetaData cfm : Schema.instance.getKeyspaceMetaData(keyspaceName).values())
ColumnFamilyStore.scrubDataDirectories(keyspaceName, cfm.cfName);
}
// clean up compaction leftovers
SetMultimap<Pair<String, String>, Integer> unfinishedCompactions = SystemKeyspace.getUnfinishedCompactions();
for (Pair<String, String> kscf : unfinishedCompactions.keySet())
{
ColumnFamilyStore.removeUnfinishedCompactionLeftovers(kscf.left, kscf.right, unfinishedCompactions.get(kscf));
}
SystemKeyspace.discardCompactionsInProgress();
// initialize keyspaces
for (String keyspaceName : Schema.instance.getKeyspaces())
{
if (logger.isDebugEnabled())
logger.debug("opening keyspace " + keyspaceName);
// disable auto compaction until commit log replay ends
for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
{
for (ColumnFamilyStore store : cfs.concatWithIndexes())
{
store.disableAutoCompaction();
}
}
}
if (CacheService.instance.keyCache.size() > 0)
logger.info("completed pre-loading ({} keys) key cache.", CacheService.instance.keyCache.size());
if (CacheService.instance.rowCache.size() > 0)
logger.info("completed pre-loading ({} keys) row cache.", CacheService.instance.rowCache.size());
try
{
GCInspector.instance.start();
}
catch (Throwable t)
{
logger.warn("Unable to start GCInspector (currently only supported on the Sun JVM)");
}
// MeteredFlusher can block if flush queue fills up, so don't put on scheduledTasks
// Start it before commit log, so memtables can flush during commit log replay
StorageService.optionalTasks.scheduleWithFixedDelay(new MeteredFlusher(), 1000, 1000, TimeUnit.MILLISECONDS);
// replay the log if necessary
try
{
CommitLog.instance.recover();
}
catch (IOException e)
{
throw new RuntimeException(e);
}
// enable auto compaction
for (Keyspace keyspace : Keyspace.all())
{
for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores())
{
for (final ColumnFamilyStore store : cfs.concatWithIndexes())
{
store.enableAutoCompaction();
}
}
}
// start compactions in five minutes (if no flushes have occurred by then to do so)
Runnable runnable = new Runnable()
{
public void run()
{
for (Keyspace keyspaceName : Keyspace.all())
{
for (ColumnFamilyStore cf : keyspaceName.getColumnFamilyStores())
{
for (ColumnFamilyStore store : cf.concatWithIndexes())
CompactionManager.instance.submitBackground(store);
}
}
}
};
StorageService.optionalTasks.schedule(runnable, 5 * 60, TimeUnit.SECONDS);
SystemKeyspace.finishStartup();
// start server internals
StorageService.instance.registerDaemon(this);
try
{
StorageService.instance.initServer();
}
catch (ConfigurationException e)
{
logger.error("Fatal configuration error", e);
System.err.println(e.getMessage() + "\nFatal configuration error; unable to start server. See log for stacktrace.");
System.exit(1);
}
Mx4jTool.maybeLoad();
// Metrics
String metricsReporterConfigFile = System.getProperty("cassandra.metricsReporterConfigFile");
if (metricsReporterConfigFile != null)
{
logger.info("Trying to load metrics-reporter-config from file: {}", metricsReporterConfigFile);
try
{
String reportFileLocation = CassandraDaemon.class.getClassLoader().getResource(metricsReporterConfigFile).getFile();
ReporterConfig.loadFromFile(reportFileLocation).enableAll();
}
catch (Exception e)
{
logger.warn("Failed to load metrics-reporter-config, metric sinks will not be activated", e);
}
}
// Thift
InetAddress rpcAddr = DatabaseDescriptor.getRpcAddress();
int rpcPort = DatabaseDescriptor.getRpcPort();
thriftServer = new ThriftServer(rpcAddr, rpcPort);
// Native transport
InetAddress nativeAddr = DatabaseDescriptor.getNativeTransportAddress();
int nativePort = DatabaseDescriptor.getNativeTransportPort();
nativeServer = new org.apache.cassandra.transport.Server(nativeAddr, nativePort);
}
/**
* Initialize the Cassandra Daemon based on the given <a
* href="http://commons.apache.org/daemon/jsvc.html">Commons
* Daemon</a>-specific arguments. To clarify, this is a hook for JSVC.
*
* @param arguments
* the arguments passed in from JSVC
* @throws IOException
*/
public void init(String[] arguments) throws IOException
{
setup();
}
/**
* Start the Cassandra Daemon, assuming that it has already been
* initialized via {@link #init(String[])}
*
* Hook for JSVC
*/
public void start()
{
String nativeFlag = System.getProperty("cassandra.start_native_transport");
if ((nativeFlag != null && Boolean.parseBoolean(nativeFlag)) || (nativeFlag == null && DatabaseDescriptor.startNativeTransport()))
nativeServer.start();
else
logger.info("Not starting native transport as requested. Use JMX (StorageService->startNativeTransport()) or nodetool (enablebinary) to start it");
String rpcFlag = System.getProperty("cassandra.start_rpc");
if ((rpcFlag != null && Boolean.parseBoolean(rpcFlag)) || (rpcFlag == null && DatabaseDescriptor.startRpc()))
thriftServer.start();
else
logger.info("Not starting RPC server as requested. Use JMX (StorageService->startRPCServer()) or nodetool (enablethrift) to start it");
}
/**
* Stop the daemon, ideally in an idempotent manner.
*
* Hook for JSVC
*/
public void stop()
{
// this doesn't entirely shut down Cassandra, just the RPC server.
// jsvc takes care of taking the rest down
logger.info("Cassandra shutting down...");
thriftServer.stop();
nativeServer.stop();
}
/**
* Clean up all resources obtained during the lifetime of the daemon. This
* is a hook for JSVC.
*/
public void destroy()
{}
/**
* A convenience method to initialize and start the daemon in one shot.
*/
public void activate()
{
String pidFile = System.getProperty("cassandra-pidfile");
try
{
try
{
MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
mbs.registerMBean(new StandardMBean(new NativeAccess(), NativeAccessMBean.class), new ObjectName(MBEAN_NAME));
}
catch (Exception e)
{
logger.error("error registering MBean " + MBEAN_NAME, e);
//Allow the server to start even if the bean can't be registered
}
setup();
if (pidFile != null)
{
new File(pidFile).deleteOnExit();
}
if (System.getProperty("cassandra-foreground") == null)
{
System.out.close();
System.err.close();
}
start();
//Start Zookeeper Service after Cassandra ~ pgaref
logger.info("Integration Starts here! ");;
//long StartTime = System.currentTimeMillis() / 1000;
try{
ZookeeperThread.start();
}catch (Throwable t){
logger.warn("Unable to start Zookeer service!");
}
/*Simple Timer
boolean timePassed = false;
while(!timePassed){
if(((System.currentTimeMillis()/1000) - StartTime) > 10)
timePassed = true;
}*/
}
catch (Throwable e)
{
logger.error("Exception encountered during startup", e);
// try to warn user on stdout too, if we haven't already detached
e.printStackTrace();
System.out.println("Exception encountered during startup: " + e.getMessage());
System.exit(3);
}
}
/**
* A convenience method to stop and destroy the daemon in one shot.
*/
public void deactivate()
{
stop();
destroy();
}
public static void stop(String[] args)
{
instance.deactivate();
}
public static void main(String[] args)
{
instance.activate();
}
static class NativeAccess implements NativeAccessMBean
{
public boolean isAvailable()
{
return CLibrary.jnaAvailable();
}
public boolean isMemoryLockable()
{
return CLibrary.jnaMemoryLockable();
}
}
public interface Server
{
/**
* Start the server.
* This method shoud be able to restart a server stopped through stop().
* Should throw a RuntimeException if the server cannot be started
*/
public void start();
/**
* Stop the server.
* This method should be able to stop server started through start().
* Should throw a RuntimeException if the server cannot be stopped
*/
public void stop();
/**
* Returns whether the server is currently running.
*/
public boolean isRunning();
}
}