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 |