Giraph Options

label type default value description
giraph.SplitMasterWorker boolean true Separate the workers and the master tasks. This is required to support dynamic recovery. (boolean)
giraph.ZkSkipAcl boolean true ZooKeeper skip ACLs
giraph.authenticate boolean false Whether to use SASL with DIGEST and Hadoop Job Tokens to authenticate and authorize Netty BSP Clients to Servers.
giraph.cleanupCheckpointsAfterSuccess boolean true Delete checkpoints after a successful job run?
giraph.createEdgeSourceVertices boolean true Create a source vertex if present in edge input but not necessarily in vertex input
giraph.disableGiraphClassResolver boolean false Disables GiraphClassResolver, which is a custom implementation of kryo class resolver that avoids writing class names to the underlying stream for faster serialization.
giraph.doOutputDuringComputation boolean false If you use this option, instead of having saving vertices in the end of application, saveVertex will be called right after each vertex.compute() is called.NOTE: This feature doesn't work well with checkpointing - if you restart from a checkpoint you won't have any ouptut from previous supresteps.
giraph.failFirstIpcPortBindAttempt boolean false Fail first IPC port binding attempt, simulate binding failure on real grid testing
giraph.failOnEmptyInput boolean true Whether to fail the job or just warn when input is empty
giraph.heap.enableReactiveJmapDumping boolean false Option to enable dumping jmap histogram reactively based on free memory on heap
giraph.isStaticGraph boolean false The application will not mutate the graph topology (the edges). It is used to optimise out-of-core graph, by not writing back edges every time.
giraph.jmap.histo.enable boolean false Configuration key to enable jmap printing
giraph.jmap.histo.live boolean false Only print live objects in jmap?
giraph.keepZooKeeperData boolean false Keep the zookeeper output for debugging? Default is to remove it.
giraph.localTestMode boolean false Indicates whether this job is run in an internal unit test
giraph.logThreadLayout boolean false Use thread level debugging?
giraph.metrics.enable boolean false Enable the Metrics system
giraph.nettyClientUseExecutionHandler boolean true Use the execution handler in netty on the client?
giraph.nettyServerUseExecutionHandler boolean true Use the execution handler in netty on the server?
giraph.nettySimulateFirstRequestClosed boolean false Netty simulate a first request closed
giraph.nettySimulateFirstResponseFailed boolean false Netty simulate a first response failed
giraph.preferIP boolean false Prefer IP addresses instead of host names
giraph.pure.yarn.job boolean false Is this a pure YARN job (i.e. no MapReduce layer managing Giraph tasks)
giraph.resendTimedOutRequests boolean true Whether to resend request which timed out or fail the job if timeout happens
giraph.textoutputformat.reverse boolean false Reverse values in the output
giraph.trackJobProgressOnClient boolean false Whether to track job progress on client or not
giraph.useBigDataIOForMessages boolean false Use BigDataIO for messages?
giraph.useInputSplitLocality boolean true To minimize network usage when reading input splits, each worker can prioritize splits that reside on its host. This, however, comes at the cost of increased load on ZooKeeper. Hence, users with a lot of splits and input threads (or with configurations that can't exploit locality) may want to disable it.
giraph.useMessageSizeEncoding boolean false Use message size encoding (typically better for complex objects, not meant for primitive wrapped messages)
giraph.useNettyDirectMemory boolean false Should netty use direct memory buffers
giraph.useNettyPooledAllocator boolean false Should netty use pooled memory allocator?
giraph.useOutOfCoreGraph boolean false Enable out-of-core graph.
giraph.useSuperstepCounters boolean true Use superstep counters? (boolean)
giraph.useUnsafeSerialization boolean true Use unsafe serialization?
giraph.vertex.resolver.create.on.msgs boolean true Option of whether to create vertexes that were not existent before but received messages
giraph.vertexOutputFormatThreadSafe boolean false Vertex output format thread-safe - if your VertexOutputFormat allows several vertexWriters to be created and written to in parallel, you should set this to true.
giraph.zKForceSync boolean false ZooKeeper force sync
giraph.zkIsExternal boolean true Zookeeper List will always hold a value during the computation while this option provides information regarding whether the zookeeper was internally started or externally provided.
giraph.aggregatorWriterClass class TextAggregatorWriter AggregatorWriter class - optional
giraph.checkpoint.supported.checker class DefaultCheckpointSupportedChecker This is the way to specify if checkpointing is supported by the job
giraph.computation.factory.class class DefaultComputationFactory Computation factory class - optional
giraph.computationClass class null Computation class - required
giraph.createEdgeSourceVerticesCallback class DefaultCreateSourceVertexCallback Decide whether we should create a source vertex when id is present in the edge input but not in vertex input
giraph.edgeInputFilterClass class DefaultEdgeInputFilter EdgeInputFilter class
giraph.edgeInputFormatClass class null EdgeInputFormat class
giraph.edgeOutputFormatClass class null EdgeOutputFormat class
giraph.edgeStoreFactoryClass class InMemoryEdgeStoreFactory Edge Store Factory class to use for creating edgeStore
giraph.edgeTranslationClass class null Class used to conduct expensive edge translation during vertex input phase
giraph.edgeValueClass class null Edge value class
giraph.edgeValueFactoryClass class DefaultEdgeValueFactory Edge value factory class - optional
giraph.gc.observers class null Classes for GC oObserver - optional
giraph.graphPartitionerFactoryClass class HashPartitionerFactory Graph partitioner factory class - optional
giraph.hadoopOutputFormatClass class BspOutputFormat Output format class for hadoop to use (for committing)
giraph.haltInstructionsWriter class DefaultHaltInstructionsWriter Class used to write instructions on how to halt the application
giraph.inputOutEdgesClass class ByteArrayEdges Vertex edges class to be used during edge input only - optional
giraph.inputOutEdgesFactoryClass class DefaultInputOutEdgesFactory OutEdges for input factory class - optional
giraph.jobObserverClass class DefaultJobObserver Observer class to watch over job status - optional
giraph.jobProgressTrackerClientClass class RetryableJobProgressTrackerClient Class to use to make calls to the job progress tracker service
giraph.jobProgressTrackerServiceClass class DefaultJobProgressTrackerService Class to use to track job progress on client
giraph.jobRetryCheckerClass class DefaultGiraphJobRetryChecker Class which decides whether a failed job should be retried - optional
giraph.mapper.observers class null Classes for Mapper Observer - optional
giraph.mappingInputFormatClass class null MappingInputFormat class
giraph.mappingStoreClass class null MappingStore Class
giraph.mappingStoreOpsClass class null MappingStoreOps class
giraph.master.observers class null Classes for Master Observer - optional
giraph.masterComputeClass class DefaultMasterCompute Class for Master - optional
giraph.messageCombinerClass class null Message combiner class - optional
giraph.messageStoreFactoryClass class InMemoryMessageStoreFactory Message Store Factory Class that is to be used
giraph.outEdgesClass class ByteArrayEdges Vertex edges class - optional
giraph.outEdgesFactoryClass class DefaultOutEdgesFactory OutEdges factory class - optional
giraph.outOfCoreDataAccessor class LocalDiskDataAccessor Data accessor used in out-of-core computation (local-disk, in-memory, HDFS, etc.)
giraph.outOfCoreOracle class MemoryEstimatorOracle Out-of-core oracle that is to be used for adaptive out-of-core engine
giraph.outgoingMessageValueClass class null Outgoing message value class
giraph.outgoingMessageValueFactoryClass class DefaultMessageValueFactory Outgoing message value factory class - optional
giraph.partitionClass class SimplePartition Partition class - optional
giraph.typesHolder class null TypesHolder, used if Computation not set - optional
giraph.vertexClass class DefaultVertex Vertex class
giraph.vertexIdClass class null Vertex index class
giraph.vertexIdFactoryClass class DefaultVertexIdFactory Vertex ID factory class - optional
giraph.vertexInputFilterClass class DefaultVertexInputFilter VertexInputFilter class
giraph.vertexInputFormatClass class null VertexInputFormat class (at least one of the input format classes is required)
giraph.vertexOutputFormatClass class null VertexOutputFormat class
giraph.vertexResolverClass class DefaultVertexResolver Vertex resolver class - optional
giraph.vertexValueClass class null Vertex value class
giraph.vertexValueCombinerClass class DefaultVertexValueCombiner Vertex value combiner class - optional
giraph.vertexValueFactoryClass class DefaultVertexValueFactory Vertex value factory class - optional
giraph.worker.observers class null Classes for Worker Observer - optional
giraph.workerContextClass class DefaultWorkerContext Worker contextclass
giraph.computation.language enum JAVA Which language computation is implemented in
giraph.messageEncodeAndStoreType enum BYTEARRAY_PER_PARTITION Select the message_encode_and_store_type to use
giraph.async.message.store.threads integer 0 Number of threads to be used in async message store.
giraph.channelsPerServer integer 1 Number of channels used per server
giraph.checkpoint.io.threads integer 8 Number of threads for writing and reading checkpoints
giraph.checkpointFrequency integer 0 How often to checkpoint (i.e. 0, means no checkpoint, 1 means every superstep, 2 is every two supersteps, etc.).
giraph.clientReceiveBufferSize integer 32768 Client receive buffer size
giraph.clientSendBufferSize integer 524288 Client send buffer size
giraph.edgeRequestSize integer 524288 Maximum size of edges (in bytes) per peer before flush
giraph.eventWaitMsecs integer 30000 Millseconds to wait for an event before continuing
giraph.hdfs.file.creation.retries integer 10 Retries to create an HDFS file before failing
giraph.hdfs.file.creation.retry.wait.ms integer 30000 Milliseconds to wait prior to retrying creation of an HDFS file
giraph.heap.minFreeMb integer 128 Option used by worker and master observers to check for imminent OOM exception
giraph.ipcInitialPort integer 30000 Initial port to start using for the IPC communication
giraph.jmap.histo.msec integer 30000 Configuration key for msec to sleep between calls
giraph.jmap.histo.print_lines integer 30 Configuration key for how many lines to print
giraph.lbMappingStoreLower integer -1 'lower' value used by lbMappingstore
giraph.lbMappingStoreUpper integer -1 'upper' value used by lbmappingstore
giraph.maxCounterWaitMsecs integer 120000 Maximum milliseconds to wait before giving up waiting forthe workers to write their counters to the zookeeper after a superstep
giraph.maxIpcPortBindAttempts integer 20 Maximum bind attempts for different IPC ports
giraph.maxMasterSuperstepWaitMsecs integer 600000 Maximum milliseconds to wait before giving up trying to get the minimum number of workers before a superstep (int).
giraph.maxMutationsPerRequest integer 100 Maximum number of mutations per partition before flush
giraph.maxNumberOfSupersteps integer 1 The application will halt after this many supersteps is completed. For instance, if it is set to 3, the application will run at most 0, 1, and 2 supersteps and then go into the shutdown superstep.
giraph.maxPartitionsInMemory integer 0 Maximum number of partitions to hold in memory for each worker. By default it is set to 0 (for adaptive out-of-core mechanism
giraph.maxRequestMilliseconds integer 600000 Milliseconds for a request to complete (or else resend)
giraph.maxResolveAddressAttempts integer 5 Max resolve address attempts
giraph.minPartitionsPerComputeThread integer 1 Minimum number of partitions to have per compute thread
giraph.msgRequestSize integer 524288 Maximum size of messages (in bytes) per peer before flush
giraph.nettyClientExecutionThreads integer 8 Netty client execution threads (execution handler)
giraph.nettyClientThreads integer 4 Netty client threads
giraph.nettyMaxConnectionFailures integer 1000 Netty max connection failures
giraph.nettyRequestEncoderBufferSize integer 32768 How big to make the encoder buffer?
giraph.nettyServerExecutionThreads integer 8 Netty server execution threads (execution handler)
giraph.nettyServerThreads integer 16 Netty server threads
giraph.numComputeThreads integer 1 Number of threads for vertex computation
giraph.numInputThreads integer 1 Number of threads for input split loading
giraph.numOutOfCoreThreads integer 1 Number of IO threads used in out-of-core mechanism. If using local disk to spill data, this should be equal to the number of available disks. In such case, use giraph.partitionsDirectory to specify mount points on different disks.
giraph.numOutputThreads integer 1 Number of threads for writing output in the end of the application
giraph.numSupersteps integer -1 Number of supersteps job will run for
giraph.partitionLongTailMinPrint integer 1 Minimum stragglers of the superstep before printing them out
giraph.serverReceiveBufferSize integer 524288 Server receive buffer size
giraph.serverSendBufferSize integer 32768 Server send buffer size
giraph.tcpBacklog integer 1 TCP backlog (defaults to number of workers)
giraph.userPartitionCount integer -1 Overrides default partition count calculation if not -1
giraph.vertexRequestSize integer 524288 Maximum size of vertices (in bytes) per peer before flush
giraph.waitTaskDoneTimeoutMs integer 900000 Maximum timeout (in ms) for waiting for all all tasks to complete
giraph.waitTimeBetweenConnectionRetriesMs integer 500
giraph.waitingRequestMsecs integer 15000 Msecs to wait between waiting for all requests to finish
giraph.yarn.task.heap.mb integer 1024 Name of Giraph property for user-configurable heap memory per worker
giraph.zKMinSessionTimeout integer 600000 ZooKeeper minimum session timeout
giraph.zkConnectionAttempts integer 10 Number of ZooKeeper client connection attempts before giving up.
giraph.zkMaxSessionTimeout integer 900000 ZooKeeper maximum session timeout
giraph.zkOpsMaxAttempts integer 3 Max attempts for handling ZooKeeper connection loss
giraph.zkOpsRetryWaitMsecs integer 5000 Msecs to wait before retrying a failed ZooKeeper op due to connection loss.
giraph.zkServerPort integer 22181 ZooKeeper port to use
giraph.zkServerlistPollMsecs integer 3000 Polling interval to check for the ZooKeeper server data
giraph.zkSessionMsecTimeout integer 60000 ZooKeeper session millisecond timeout
mapred.map.max.attempts integer -1 Maximum number of attempts a master/worker will retry before killing the job. This directly maps to the number of map task attempts in Hadoop.
giraph.additionalEdgeRequestSize float 0.2 Additional size (expressed as a ratio) of each per-partition buffer on top of the average size.
giraph.additionalMsgRequestSize float 0.2 How much bigger than the average per partition size to make initial per partition buffers. If this value is A, message request size is M, and a worker has P partitions, than its initial partition buffer size will be (M / P) * (1 + A).
giraph.additionalVertexRequestSize float 0.2 Additional size (expressed as a ratio) of each per-partition buffer on top of the average size.
giraph.inputSplitSamplePercent float 100.0 Input split sample percent - Used only for sampling and testing, rather than an actual job. The idea is that to test, you might only want a fraction of the actual input splits from your VertexInputFormat to load (values should be [0, 100]).
giraph.masterPartitionCountMultiplier float 1.0 Multiplier for the current workers squared
giraph.minPercentResponded float 100.0 Minimum percent of the maximum number of workers that have responded in order to continue progressing. (float)
giraph.msgRequestWarningThreshold float 2.0 If request sizes are bigger than the buffer size by this factor warnings are printed to the log and to the command line
giraph.InputSplitMaxEdges long -1 To limit outlier vertex input splits from producing too many vertices or to help with testing, the number of edges loaded from an input split can be limited. By default, everything is loaded.
giraph.InputSplitMaxVertices long -1 To limit outlier vertex input splits from producing too many vertices or to help with testing, the number of vertices loaded from an input split can be limited. By default, everything is loaded.
giraph.VerticesToUpdateProgress long 100000 Minimum number of vertices to compute before updating worker progress
giraph.maxAllowedJobTimeMilliseconds long -1 Maximum allowed time for job to run after getting all resources before it will be killed, in milliseconds (-1 if it has no limit)
giraph.waitForOtherWorkersMsec long 172800000 How long should workers wait to finish superstep
giraph.waitZookeeperTimeoutMsec long 900000 How long should we stay in waitForever loops in various places that require network connection
giraph.checkpoint.compression.codec string .deflate Defines compression algorithm we will be using for storing checkpoint. Available options include but not restricted to: .deflate, .gz, .bz2, .lzo
giraph.checkpointDirectory string _bsp/_checkpoints/ This directory has/stores the available checkpoint files in HDFS.
giraph.dns.interface string default Interface to use for hostname resolution
giraph.dns.nameserver string default Server for hostname resolution
giraph.edge.output.subdir string EdgeOutputFormat sub-directory
giraph.jmapPath string jmap Path to use for invoking jmap
giraph.logLevel string info Override the Hadoop log level and set the desired log level.
giraph.metrics.directory string Directory in HDFS to write master metrics to, instead of stderr
giraph.nettyClientExecutionAfterHandler string request-encoder Where to place the netty client execution handle?
giraph.nettyCompressionAlgorithm string Which compression algorithm to use in netty
giraph.nettyServerExecutionAfterHandler string requestFrameDecoder Where to place the netty server execution handle?
giraph.partitionsDirectory string _bsp/_partitions Comma-separated list of directories in the local filesystem for out-of-core partitions.
giraph.restart.jobId string null Which job ID should I try to restart?
giraph.textoutputformat.separator string GiraphTextOuputFormat Separator
giraph.vertex.output.subdir string VertexOutputFormat sub-directory
giraph.yarn.libjars string conf key for comma-separated list of jars to export to YARN workers
giraph.zkList string ZooKeeper comma-separated list (if not set, will start up ZooKeeper locally). Consider that after locally-starting zookeeper, this parameter will updated the configuration with the corrent configuration value.
giraph.zkManagerDirectory string _bsp/_defaultZkManagerDir If ZOOKEEPER_LIST is not set, then use this directory to manage ZooKeeper