├── CHANGES.txt
├── LICENSE.txt
├── NOTICE.txt
├── README.md
├── bin
├── grooms.sh
├── hama
├── hama-config.sh
├── hama-daemon.sh
├── hama-daemons.sh
├── start-bspd.sh
├── stop-bspd.sh
└── zookeepers.sh
├── c++
├── README.txt
├── pom.xml
└── src
│ ├── CMakeLists.txt
│ ├── JNIFlags.cmake
│ └── main
│ └── native
│ ├── examples
│ ├── README.txt
│ ├── conf
│ │ ├── matrixmultiplication.xml
│ │ ├── piestimator.xml
│ │ └── summation.xml
│ └── impl
│ │ ├── DenseDoubleVector.cc
│ │ ├── DenseDoubleVector.hh
│ │ ├── matrixmultiplication.cc
│ │ ├── piestimator.cc
│ │ └── summation.cc
│ ├── pipes
│ ├── api
│ │ └── hama
│ │ │ ├── Pipes.hh
│ │ │ └── TemplateFactory.hh
│ └── impl
│ │ └── Pipes.cc
│ └── utils
│ ├── api
│ └── hadoop
│ │ ├── SerialUtils.hh
│ │ ├── Splitter.hh
│ │ └── StringUtils.hh
│ └── impl
│ ├── SerialUtils.cc
│ ├── Splitter.cc
│ └── StringUtils.cc
├── commons
├── pom.xml
└── src
│ ├── main
│ └── java
│ │ └── org
│ │ └── apache
│ │ └── hama
│ │ └── commons
│ │ ├── io
│ │ ├── DenseVectorWritable.java
│ │ ├── FloatArrayWritable.java
│ │ ├── FloatMatrixWritable.java
│ │ ├── FloatVectorWritable.java
│ │ ├── KeyValueWritable.java
│ │ ├── MatrixWritable.java
│ │ ├── PipesKeyValueWritable.java
│ │ ├── PipesVectorWritable.java
│ │ ├── SparseVectorWritable.java
│ │ ├── StringArrayWritable.java
│ │ ├── TextArrayWritable.java
│ │ └── VectorWritable.java
│ │ ├── math
│ │ ├── DenseDoubleMatrix.java
│ │ ├── DenseDoubleVector.java
│ │ ├── DenseFloatMatrix.java
│ │ ├── DenseFloatVector.java
│ │ ├── DoubleDoubleFunction.java
│ │ ├── DoubleDoubleVectorFunction.java
│ │ ├── DoubleFunction.java
│ │ ├── DoubleMatrix.java
│ │ ├── DoubleVector.java
│ │ ├── DoubleVectorFunction.java
│ │ ├── FloatFloatFunction.java
│ │ ├── FloatFunction.java
│ │ ├── FloatMatrix.java
│ │ ├── FloatVector.java
│ │ ├── Function.java
│ │ ├── NamedDoubleVector.java
│ │ ├── NamedFloatVector.java
│ │ ├── SparseDoubleVector.java
│ │ ├── SquareVectorFunction.java
│ │ └── Tuple.java
│ │ └── util
│ │ ├── KeyValuePair.java
│ │ └── TextPair.java
│ └── test
│ └── java
│ └── org
│ └── apache
│ └── hama
│ └── commons
│ └── math
│ ├── TestDenseDoubleMatrix.java
│ ├── TestDenseDoubleVector.java
│ └── TestSparseDoubleVector.java
├── conf
├── groomservers
├── hama-default.xml
├── hama-env.sh
├── hama-site.xml
└── log4j.properties
├── contrib
├── avro-rpc
│ └── src
│ │ └── main
│ │ └── java
│ │ └── org
│ │ └── apache
│ │ └── hama
│ │ └── avro
│ │ ├── AvroBSPMessageBundle.java
│ │ ├── AvroMessageManagerImpl.java
│ │ ├── Sender.java
│ │ └── TestAvroMessageManager.java
└── monitor-plugin
│ └── jvm-metrics
│ ├── pom.xml
│ └── src
│ └── main
│ └── java
│ └── org
│ └── apache
│ └── hama
│ └── monitor
│ └── plugin
│ └── JvmTask.java
├── core
├── dev-support
│ └── saveVersion.sh
├── pom.xml
└── src
│ ├── main
│ ├── java
│ │ └── org
│ │ │ └── apache
│ │ │ ├── hama
│ │ │ ├── BSPMasterRunner.java
│ │ │ ├── Constants.java
│ │ │ ├── GroomServerRunner.java
│ │ │ ├── HamaConfiguration.java
│ │ │ ├── HamaVersionAnnotation.java
│ │ │ ├── ZooKeeperRunner.java
│ │ │ ├── bsp
│ │ │ │ ├── BSP.java
│ │ │ │ ├── BSPInterface.java
│ │ │ │ ├── BSPJob.java
│ │ │ │ ├── BSPJobClient.java
│ │ │ │ ├── BSPJobContext.java
│ │ │ │ ├── BSPJobID.java
│ │ │ │ ├── BSPMaster.java
│ │ │ │ ├── BSPMessageBundle.java
│ │ │ │ ├── BSPMessageBundleInterface.java
│ │ │ │ ├── BSPPeer.java
│ │ │ │ ├── BSPPeerImpl.java
│ │ │ │ ├── BSPTask.java
│ │ │ │ ├── BSPTaskRunner.java
│ │ │ │ ├── ClusterStatus.java
│ │ │ │ ├── CombineFileInputFormat.java
│ │ │ │ ├── CombineFileSplit.java
│ │ │ │ ├── Combiner.java
│ │ │ │ ├── CommitTaskAction.java
│ │ │ │ ├── Counters.java
│ │ │ │ ├── Directive.java
│ │ │ │ ├── DirectiveException.java
│ │ │ │ ├── DirectiveHandler.java
│ │ │ │ ├── DispatchTasksDirective.java
│ │ │ │ ├── FCFSQueue.java
│ │ │ │ ├── FileInputFormat.java
│ │ │ │ ├── FileOutputFormat.java
│ │ │ │ ├── FileSplit.java
│ │ │ │ ├── GroomServer.java
│ │ │ │ ├── GroomServerAction.java
│ │ │ │ ├── GroomServerManager.java
│ │ │ │ ├── GroomServerStatus.java
│ │ │ │ ├── GroomStatusListener.java
│ │ │ │ ├── HashPartitioner.java
│ │ │ │ ├── ID.java
│ │ │ │ ├── InputFormat.java
│ │ │ │ ├── InputSplit.java
│ │ │ │ ├── JobChangeEvent.java
│ │ │ │ ├── JobInProgress.java
│ │ │ │ ├── JobInProgressListener.java
│ │ │ │ ├── JobProfile.java
│ │ │ │ ├── JobStatus.java
│ │ │ │ ├── KeyValueLineRecordReader.java
│ │ │ │ ├── KeyValueTextInputFormat.java
│ │ │ │ ├── KillJobAction.java
│ │ │ │ ├── KillTaskAction.java
│ │ │ │ ├── LaunchTaskAction.java
│ │ │ │ ├── LineRecordReader.java
│ │ │ │ ├── LocalBSPRunner.java
│ │ │ │ ├── Messagable.java
│ │ │ │ ├── MonitorManager.java
│ │ │ │ ├── NonSplitSequenceFileInputFormat.java
│ │ │ │ ├── NullInputFormat.java
│ │ │ │ ├── NullOutputFormat.java
│ │ │ │ ├── OutputCollector.java
│ │ │ │ ├── OutputFormat.java
│ │ │ │ ├── Partitioner.java
│ │ │ │ ├── PartitioningRunner.java
│ │ │ │ ├── Queue.java
│ │ │ │ ├── QueueManager.java
│ │ │ │ ├── RecordReader.java
│ │ │ │ ├── RecordWriter.java
│ │ │ │ ├── RecoverTaskAction.java
│ │ │ │ ├── ReinitGroomAction.java
│ │ │ │ ├── ReportGroomStatusDirective.java
│ │ │ │ ├── RunningJob.java
│ │ │ │ ├── Schedulable.java
│ │ │ │ ├── SequenceFileInputFormat.java
│ │ │ │ ├── SequenceFileOutputFormat.java
│ │ │ │ ├── SequenceFileRecordReader.java
│ │ │ │ ├── SequenceFileRecordWriter.java
│ │ │ │ ├── SimpleTaskScheduler.java
│ │ │ │ ├── SimpleTaskWorkerManager.java
│ │ │ │ ├── Superstep.java
│ │ │ │ ├── SuperstepBSP.java
│ │ │ │ ├── Task.java
│ │ │ │ ├── TaskAttemptContext.java
│ │ │ │ ├── TaskAttemptID.java
│ │ │ │ ├── TaskCompletionEvent.java
│ │ │ │ ├── TaskID.java
│ │ │ │ ├── TaskInProgress.java
│ │ │ │ ├── TaskLog.java
│ │ │ │ ├── TaskLogAppender.java
│ │ │ │ ├── TaskLogServlet.java
│ │ │ │ ├── TaskRunner.java
│ │ │ │ ├── TaskScheduler.java
│ │ │ │ ├── TaskStatus.java
│ │ │ │ ├── TaskWorkerManager.java
│ │ │ │ ├── TextInputFormat.java
│ │ │ │ ├── TextOutputFormat.java
│ │ │ │ ├── TrackedRecordReader.java
│ │ │ │ ├── UpdatePeerAction.java
│ │ │ │ ├── ft
│ │ │ │ │ ├── AsyncRcvdMsgCheckpointImpl.java
│ │ │ │ │ ├── BSPFaultTolerantService.java
│ │ │ │ │ ├── FaultTolerantMasterService.java
│ │ │ │ │ └── FaultTolerantPeerService.java
│ │ │ │ ├── join
│ │ │ │ │ ├── ComposableInputFormat.java
│ │ │ │ │ ├── ComposableRecordReader.java
│ │ │ │ │ ├── CompositeInputFormat.java
│ │ │ │ │ ├── CompositeInputSplit.java
│ │ │ │ │ ├── CompositeRecordReader.java
│ │ │ │ │ ├── InnerJoinRecordReader.java
│ │ │ │ │ ├── JoinRecordReader.java
│ │ │ │ │ ├── MultiFilterRecordReader.java
│ │ │ │ │ ├── OuterJoinRecordReader.java
│ │ │ │ │ ├── OverrideRecordReader.java
│ │ │ │ │ ├── Parser.java
│ │ │ │ │ ├── ResetableIterator.java
│ │ │ │ │ ├── StreamBackedIterator.java
│ │ │ │ │ ├── TupleWritable.java
│ │ │ │ │ └── WrappedRecordReader.java
│ │ │ │ ├── message
│ │ │ │ │ ├── AbstractMessageManager.java
│ │ │ │ │ ├── AbstractOutgoingMessageManager.java
│ │ │ │ │ ├── HamaAsyncMessageManagerImpl.java
│ │ │ │ │ ├── HamaMessageManager.java
│ │ │ │ │ ├── HamaMessageManagerImpl.java
│ │ │ │ │ ├── MessageEventListener.java
│ │ │ │ │ ├── MessageManager.java
│ │ │ │ │ ├── MessageManagerFactory.java
│ │ │ │ │ ├── OutgoingMessageManager.java
│ │ │ │ │ ├── OutgoingPOJOMessageBundle.java
│ │ │ │ │ ├── compress
│ │ │ │ │ │ ├── BSPMessageCompressor.java
│ │ │ │ │ │ ├── BSPMessageCompressorFactory.java
│ │ │ │ │ │ ├── Bzip2Compressor.java
│ │ │ │ │ │ └── package.html
│ │ │ │ │ ├── package.html
│ │ │ │ │ └── queue
│ │ │ │ │ │ ├── MemoryQueue.java
│ │ │ │ │ │ ├── MessageQueue.java
│ │ │ │ │ │ ├── SingleLockQueue.java
│ │ │ │ │ │ ├── SortedMemoryQueue.java
│ │ │ │ │ │ ├── SynchronizedQueue.java
│ │ │ │ │ │ └── package.html
│ │ │ │ ├── package.html
│ │ │ │ ├── sync
│ │ │ │ │ ├── BSPMasterSyncClient.java
│ │ │ │ │ ├── BSPPeerSyncClient.java
│ │ │ │ │ ├── MasterSyncClient.java
│ │ │ │ │ ├── PeerSyncClient.java
│ │ │ │ │ ├── SyncClient.java
│ │ │ │ │ ├── SyncEvent.java
│ │ │ │ │ ├── SyncEventListener.java
│ │ │ │ │ ├── SyncException.java
│ │ │ │ │ ├── SyncServer.java
│ │ │ │ │ ├── SyncServerRunner.java
│ │ │ │ │ ├── SyncServiceFactory.java
│ │ │ │ │ ├── ZKSyncBSPMasterClient.java
│ │ │ │ │ ├── ZKSyncClient.java
│ │ │ │ │ ├── ZKSyncEventFactory.java
│ │ │ │ │ ├── ZKSyncEventListener.java
│ │ │ │ │ ├── ZooKeeperSyncClientImpl.java
│ │ │ │ │ ├── ZooKeeperSyncServerImpl.java
│ │ │ │ │ └── package.html
│ │ │ │ └── taskallocation
│ │ │ │ │ ├── BSPResource.java
│ │ │ │ │ ├── BestEffortDataLocalTaskAllocator.java
│ │ │ │ │ ├── RawSplitResource.java
│ │ │ │ │ ├── RoundRobinTaskAllocator.java
│ │ │ │ │ └── TaskAllocationStrategy.java
│ │ │ ├── http
│ │ │ │ ├── HttpServer.java
│ │ │ │ └── package.html
│ │ │ ├── ipc
│ │ │ │ ├── AsyncClient.java
│ │ │ │ ├── AsyncRPC.java
│ │ │ │ ├── AsyncServer.java
│ │ │ │ ├── BSPPeerProtocol.java
│ │ │ │ ├── Client.java
│ │ │ │ ├── ConnectionHeader.java
│ │ │ │ ├── GroomProtocol.java
│ │ │ │ ├── HamaRPCProtocolVersion.java
│ │ │ │ ├── JobSubmissionProtocol.java
│ │ │ │ ├── MasterProtocol.java
│ │ │ │ ├── RPC.java
│ │ │ │ ├── RemoteException.java
│ │ │ │ ├── RetryPolicies.java
│ │ │ │ ├── RetryPolicy.java
│ │ │ │ ├── Server.java
│ │ │ │ ├── Status.java
│ │ │ │ ├── VersionedProtocol.java
│ │ │ │ └── package.html
│ │ │ ├── manager
│ │ │ │ ├── LogView.java
│ │ │ │ └── util
│ │ │ │ │ └── UITemplate.java
│ │ │ ├── monitor
│ │ │ │ ├── Configurator.java
│ │ │ │ ├── Federator.java
│ │ │ │ ├── Metric.java
│ │ │ │ ├── MetricsRecord.java
│ │ │ │ ├── MetricsTag.java
│ │ │ │ ├── Monitor.java
│ │ │ │ ├── MonitorListener.java
│ │ │ │ ├── ZKCollector.java
│ │ │ │ └── fd
│ │ │ │ │ ├── FDProvider.java
│ │ │ │ │ ├── NodeEventListener.java
│ │ │ │ │ ├── NodeStatus.java
│ │ │ │ │ ├── Sensor.java
│ │ │ │ │ ├── Supervisor.java
│ │ │ │ │ ├── UDPSensor.java
│ │ │ │ │ └── UDPSupervisor.java
│ │ │ ├── package.html
│ │ │ ├── pipes
│ │ │ │ ├── PipesApplication.java
│ │ │ │ ├── PipesBSP.java
│ │ │ │ ├── PipesNonJavaInputFormat.java
│ │ │ │ ├── PipesPartitioner.java
│ │ │ │ ├── Submitter.java
│ │ │ │ ├── protocol
│ │ │ │ │ ├── BinaryProtocol.java
│ │ │ │ │ ├── DownwardProtocol.java
│ │ │ │ │ ├── MessageType.java
│ │ │ │ │ ├── StreamingProtocol.java
│ │ │ │ │ └── UplinkReader.java
│ │ │ │ └── util
│ │ │ │ │ ├── DistributedCacheUtil.java
│ │ │ │ │ └── SequenceFileDumper.java
│ │ │ ├── util
│ │ │ │ ├── BSPNetUtils.java
│ │ │ │ ├── BSPServletUtil.java
│ │ │ │ ├── ByteUtils.java
│ │ │ │ ├── Bytes.java
│ │ │ │ ├── ClusterUtil.java
│ │ │ │ ├── DistCacheUtils.java
│ │ │ │ ├── ExtendedDataOutput.java
│ │ │ │ ├── LRUCache.java
│ │ │ │ ├── ProgramDriver.java
│ │ │ │ ├── RandomVariable.java
│ │ │ │ ├── ReflectionUtils.java
│ │ │ │ ├── RunJar.java
│ │ │ │ ├── SocketIOWithTimeout.java
│ │ │ │ ├── SocketInputStream.java
│ │ │ │ ├── SocketOutputStream.java
│ │ │ │ ├── UnsafeByteArrayInputStream.java
│ │ │ │ ├── UnsafeByteArrayOutputStream.java
│ │ │ │ ├── VersionInfo.java
│ │ │ │ ├── WritableUtils.java
│ │ │ │ ├── ZKUtil.java
│ │ │ │ └── package.html
│ │ │ └── zookeeper
│ │ │ │ ├── QuorumPeer.java
│ │ │ │ ├── ZKServerTool.java
│ │ │ │ └── package.html
│ │ │ └── overview.html
│ └── resources
│ │ └── webapp
│ │ ├── bspmaster
│ │ ├── bspjob.jsp
│ │ ├── bspmaster.jsp
│ │ ├── index.html
│ │ └── machines.jsp
│ │ ├── commons
│ │ └── tpl
│ │ │ ├── tpl.configlist.html
│ │ │ └── tpl.logview.html
│ │ ├── groomserver
│ │ ├── groomserver.jsp
│ │ └── index.html
│ │ └── static
│ │ └── hama.css
│ └── test
│ ├── java
│ └── org
│ │ └── apache
│ │ └── hama
│ │ ├── HamaCluster.java
│ │ ├── HamaClusterTestCase.java
│ │ ├── HamaTestCase.java
│ │ ├── MiniBSPCluster.java
│ │ ├── MiniZooKeeperCluster.java
│ │ ├── bsp
│ │ ├── TestAdditionalTasks.java
│ │ ├── TestBSPMasterGroomServer.java
│ │ ├── TestBSPMessageBundle.java
│ │ ├── TestBSPTaskFaults.java
│ │ ├── TestCheckpoint.java
│ │ ├── TestClusterStatus.java
│ │ ├── TestCompositeInputFormat.java
│ │ ├── TestFileInputFormat.java
│ │ ├── TestKeyValueTextInputFormat.java
│ │ ├── TestLocalRunner.java
│ │ ├── TestPartitioning.java
│ │ ├── TestPersistQueue.java
│ │ ├── TestTaskAllocation.java
│ │ ├── TestTaskAllocationRoundRobin.java
│ │ ├── TestZooKeeper.java
│ │ ├── message
│ │ │ ├── TestHamaAsyncMessageManager.java
│ │ │ ├── TestHamaMessageManager.java
│ │ │ └── compress
│ │ │ │ └── TestBSPMessageCompressor.java
│ │ └── sync
│ │ │ └── TestSyncServiceFactory.java
│ │ ├── examples
│ │ └── ClassSerializePrinting.java
│ │ ├── ipc
│ │ ├── TestAsyncIPC.java
│ │ ├── TestAsyncRPC.java
│ │ ├── TestIPC.java
│ │ └── TestRPC.java
│ │ ├── manager
│ │ ├── TestLogView.java
│ │ └── util
│ │ │ └── TestUITemplate.java
│ │ ├── monitor
│ │ ├── TestConfigurator.java
│ │ ├── TestFederator.java
│ │ └── fd
│ │ │ └── TestFD.java
│ │ ├── pipes
│ │ └── TestPipes.java
│ │ ├── util
│ │ ├── TestBytes.java
│ │ ├── TestNumeric.java
│ │ ├── TestRandomVariable.java
│ │ └── TestZKUtil.java
│ │ └── zookeeper
│ │ └── TestZKTools.java
│ └── resources
│ ├── hama-metrics-msys.properties
│ └── hama-metrics-test-config.properties
├── dist
└── pom.xml
├── docker
├── Dockerfile
└── README.md
├── docs
└── diagram
│ ├── hama-diagram.pptx
│ └── yarn-hama-diagram.pptx
├── examples
├── pom.xml
└── src
│ ├── main
│ └── java
│ │ └── org
│ │ └── apache
│ │ └── hama
│ │ └── examples
│ │ ├── BipartiteMatching.java
│ │ ├── CombineExample.java
│ │ ├── DynamicGraph.java
│ │ ├── ExampleDriver.java
│ │ ├── GradientDescentExample.java
│ │ ├── InlinkCount.java
│ │ ├── KCore.java
│ │ ├── Kmeans.java
│ │ ├── MindistSearch.java
│ │ ├── PageRank.java
│ │ ├── PiEstimator.java
│ │ ├── RandBench.java
│ │ ├── SSSP.java
│ │ ├── SemiClusterJobDriver.java
│ │ ├── SpMV.java
│ │ ├── SuperstepPiEstimator.java
│ │ └── util
│ │ ├── FastGraphGen.java
│ │ ├── Generator.java
│ │ ├── SymmetricMatrixGen.java
│ │ └── VectorWritableMatrixGen.java
│ └── test
│ ├── java
│ └── org
│ │ └── apache
│ │ └── hama
│ │ └── examples
│ │ ├── BipartiteMatchingTest.java
│ │ ├── CombineExampleTest.java
│ │ ├── CustomVertexReadWriteStateTest.java
│ │ ├── DynamicGraphTest.java
│ │ ├── FastGraphGenTest.java
│ │ ├── KCoreTest.java
│ │ ├── LinearRegressionTest.java
│ │ ├── LogisticRegressionTest.java
│ │ ├── MindistSearchTest.java
│ │ ├── PageRankTest.java
│ │ ├── PiEstimatorTest.java
│ │ ├── RandBenchTest.java
│ │ ├── SSSPTest.java
│ │ ├── SemiClusterMatchingTest.java
│ │ ├── SpMVTest.java
│ │ └── SymmetricMatrixGenTest.java
│ └── resources
│ ├── dg.txt
│ ├── kcore.txt
│ ├── linear_regression_sample.txt
│ ├── logistic_regression_sample.txt
│ └── semiclustering.txt
├── graph
├── pom.xml
└── src
│ ├── main
│ └── java
│ │ └── org
│ │ └── apache
│ │ └── hama
│ │ └── graph
│ │ ├── AbsDiffAggregator.java
│ │ ├── AbstractAggregator.java
│ │ ├── AggregationRunner.java
│ │ ├── Aggregator.java
│ │ ├── AverageAggregator.java
│ │ ├── DefaultVertexOutputWriter.java
│ │ ├── Edge.java
│ │ ├── GraphJob.java
│ │ ├── GraphJobMessage.java
│ │ ├── GraphJobRunner.java
│ │ ├── IncomingVertexMessageManager.java
│ │ ├── MapVerticesInfo.java
│ │ ├── MaxAggregator.java
│ │ ├── MinAggregator.java
│ │ ├── OutgoingVertexMessageManager.java
│ │ ├── SumAggregator.java
│ │ ├── Vertex.java
│ │ ├── VertexInputReader.java
│ │ ├── VertexInterface.java
│ │ ├── VertexOutputWriter.java
│ │ ├── VerticesInfo.java
│ │ └── package.html
│ └── test
│ └── java
│ └── org
│ └── apache
│ └── hama
│ └── graph
│ ├── TestAbsDiffAggregator.java
│ ├── TestAverageAggregator.java
│ ├── TestGraphJobMessage.java
│ ├── TestMinMaxAggregator.java
│ ├── TestSubmitGraphJob.java
│ ├── TestSumAggregator.java
│ └── example
│ └── PageRank.java
├── mesos
├── .gitignore
├── pom.xml
└── src
│ └── main
│ └── java
│ └── org
│ └── apache
│ └── hama
│ └── bsp
│ ├── MesosExecutor.java
│ ├── MesosScheduler.java
│ ├── ResourceManager.java
│ └── TaskDelegator.java
├── ml
├── pom.xml
└── src
│ ├── main
│ └── java
│ │ └── org
│ │ └── apache
│ │ └── hama
│ │ └── ml
│ │ ├── classification
│ │ ├── ClassifierAdapter.java
│ │ └── ClassifierConfigurationAdapter.java
│ │ ├── distance
│ │ ├── CosineDistance.java
│ │ ├── DistanceMeasurer.java
│ │ └── EuclidianDistance.java
│ │ ├── kcore
│ │ ├── KCoreMessage.java
│ │ ├── KCoreVertex.java
│ │ ├── KCoreVertexReader.java
│ │ └── KCoreVertexWriter.java
│ │ ├── kmeans
│ │ ├── CenterMessage.java
│ │ └── KMeansBSP.java
│ │ ├── recommendation
│ │ ├── ItemSimilarity.java
│ │ ├── Preference.java
│ │ ├── Recommender.java
│ │ ├── RecommenderIO.java
│ │ ├── UserSimilarity.java
│ │ └── cf
│ │ │ ├── InputConverter.java
│ │ │ ├── KeyValueParser.java
│ │ │ ├── MovieLensConverter.java
│ │ │ ├── OnlineCF.java
│ │ │ ├── OnlineTrainBSP.java
│ │ │ └── function
│ │ │ ├── MeanAbsError.java
│ │ │ └── OnlineUpdate.java
│ │ ├── regression
│ │ ├── CostFunction.java
│ │ ├── GradientDescentBSP.java
│ │ ├── HypothesisFunction.java
│ │ ├── LinearRegressionModel.java
│ │ ├── LogisticRegressionModel.java
│ │ ├── RegressionModel.java
│ │ └── VectorDoubleFileInputFormat.java
│ │ ├── semiclustering
│ │ ├── SemiClusterDetails.java
│ │ ├── SemiClusterMessage.java
│ │ ├── SemiClusterTextReader.java
│ │ ├── SemiClusterVertexOutputWriter.java
│ │ └── SemiClusteringVertex.java
│ │ └── util
│ │ ├── DefaultFeatureTransformer.java
│ │ └── FeatureTransformer.java
│ └── test
│ ├── java
│ └── org
│ │ └── apache
│ │ └── hama
│ │ └── ml
│ │ ├── MLTestBase.java
│ │ ├── kmeans
│ │ └── TestKMeansBSP.java
│ │ ├── recommendation
│ │ └── TestOnlineCF.java
│ │ └── regression
│ │ ├── LinearRegressionModelTest.java
│ │ ├── LogisticRegressionModelTest.java
│ │ └── VectorDoubleFileInputFormatTest.java
│ └── resources
│ ├── dimensional_reduction.txt
│ ├── linear_regression_data.txt
│ ├── logistic_regression_data.txt
│ └── vd_file_sample.txt
├── pom.xml
├── python
├── BSP.py
├── BSPPeer.py
├── BSPRunner.py
├── BinaryProtocol.py
├── BspJobConfiguration.py
├── HelloWorldBSP.py
├── KMeansBSP.py
└── README.txt
├── src
├── assemble
│ ├── bin.xml
│ └── src.xml
└── site
│ ├── resources
│ ├── css
│ │ └── site.css
│ ├── doap_Hama.rdf
│ ├── files
│ │ └── hama-eclipse-formatter.xml
│ └── images
│ │ ├── apache-hama-white.png
│ │ ├── favicon.ico
│ │ ├── hama_art_arthur.png
│ │ ├── hama_art_arthur_300x220.png
│ │ ├── hama_kim.png
│ │ ├── hama_logo.png
│ │ ├── hama_paint_logo.png
│ │ ├── hama_paint_logo_120x120.png
│ │ ├── headerlogo.png
│ │ ├── headerlogo.xcf
│ │ └── mahout_vs_hama.png
│ ├── site.xml
│ └── xdoc
│ ├── artwork.xml
│ ├── developers.xml
│ ├── downloads.xml
│ ├── getting_started_with_hama.xml
│ ├── hama_bsp_tutorial.xml
│ ├── hama_graph_tutorial.xml
│ ├── index.xml
│ ├── irc-channel.xml
│ ├── issue-tracking.xml
│ ├── mail-lists.xml
│ ├── privacy_policy.xml
│ ├── run_examples.xml
│ └── team-list.xml
└── yarn
├── pom.xml
└── src
└── main
├── java
└── org
│ └── apache
│ └── hama
│ └── bsp
│ ├── ApplicationMaster.java
│ ├── BSPClient.java
│ ├── BSPRunner.java
│ ├── PageRankonYarn.java
│ ├── YARNBSPConstants.java
│ ├── YARNBSPJob.java
│ ├── YARNBSPJobClient.java
│ ├── YARNGraphJob.java
│ └── YarnSerializePrinting.java
└── resources
└── log4j.properties
/NOTICE.txt:
--------------------------------------------------------------------------------
1 | Apache Hama
2 | Copyright 2008-2015 The Apache Software Foundation
3 |
4 | This product includes software developed at
5 | The Apache Software Foundation (http://www.apache.org/).
6 |
7 | This product includes JUnit under the Common Public License Version 1.0
8 | See http://www.junit.org/
9 |
10 | This product includes SLF4J software developed by QOS.ch.
11 | Copyright c 2004-2008 QOS.ch. All rights reserved.
12 |
13 | The javax.servlet package used by Jetty is copyright
14 | Sun Microsystems, Inc and Apache Software Foundation. It is
15 | distributed under the Common Development and Distribution License.
16 | See https://glassfish.dev.java.net/public/CDDLv1.0.html.
17 |
18 | Compression support provided by snappy-java
19 | See http://code.google.com/p/snappy-java/
20 |
21 | This product includes software developed by the Google Guava project.
22 | Copyright (C) 2009 Google Inc.
23 | See http://code.google.com/p/guava-libraries/
24 |
25 | This product includes software developed by the Jackson project.
26 | Copyright 2010 FasterXML, LLC
27 | See http://jackson.codehaus.org/
28 |
29 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # Apache Hama
2 |
3 | Apache Hama is a framework for Big Data analytics which uses the Bulk Synchronous Parallel (BSP) computing model, which was established in 2012 as a Top-Level Project of The Apache Software Foundation.
4 |
5 | It provides not only pure BSP programming model but also SQL-like query interface (Apache MRQL) and vertex/neuron centric programming models, inspired by Google's Pregel and DistBelief (Apache Horn). For the latest information about Hama, please visit our website at: and our wiki at:
6 |
7 | ## Getting Started
8 |
9 | Please refer to the [Installation Guide](http://wiki.apache.org/hama/GettingStarted) in the online documentation for an overview on how to getting started with Hama.
10 |
11 | ## Run Examples
12 |
13 | Hama provides examples package that allows you to quickly run examples on your Hama Cluster. To run one of them, use `% $HAMA_HOME/bin/hama jar hama-examples-x.x.x.jar`. For example:
14 |
15 | Download a [Iris dataset](http://people.apache.org/~edwardyoon/kmeans.txt). And then, run K-Means using:
16 |
17 | `% $HAMA_HOME/bin/hama jar hama-examples-x.x.x.jar kmeans /tmp/kmeans.txt /tmp/result 10 3`
18 |
19 | ## Getting Involved
20 |
21 | Hama is an open source volunteer project under the Apache Software Foundation. We encourage you to learn about the project and contribute your expertise.
22 |
--------------------------------------------------------------------------------
/bin/hama-daemons.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 |
3 | # Licensed to the Apache Software Foundation (ASF) under one or more
4 | # contributor license agreements. See the NOTICE file distributed with
5 | # this work for additional information regarding copyright ownership.
6 | # The ASF licenses this file to You under the Apache License, Version 2.0
7 | # (the "License"); you may not use this file except in compliance with
8 | # the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 |
18 |
19 | # Run a Hama command on all slave hosts.
20 |
21 | usage="Usage: hama-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command args..."
22 |
23 | # if no args specified, show usage
24 | if [ $# -le 1 ]; then
25 | echo $usage
26 | exit 1
27 | fi
28 |
29 | bin=`dirname "$0"`
30 | bin=`cd "$bin"; pwd`
31 |
32 | . $bin/hama-config.sh
33 |
34 | remote_cmd="cd ${HAMA_HOME}; $bin/hama-daemon.sh --config ${HAMA_CONF_DIR} $@"
35 | args="--config ${HAMA_CONF_DIR} $remote_cmd"
36 | command=$2
37 |
38 | case $command in
39 | (zookeeper)
40 | exec "$bin/zookeepers.sh" $args
41 | ;;
42 | (*)
43 | exec "$bin/grooms.sh" $args
44 | ;;
45 | esac
--------------------------------------------------------------------------------
/bin/start-bspd.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 |
3 | # Licensed to the Apache Software Foundation (ASF) under one or more
4 | # contributor license agreements. See the NOTICE file distributed with
5 | # this work for additional information regarding copyright ownership.
6 | # The ASF licenses this file to You under the Apache License, Version 2.0
7 | # (the "License"); you may not use this file except in compliance with
8 | # the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 |
18 |
19 | # Start hama map reduce daemons. Run this on master node.
20 |
21 | bin=`dirname "$0"`
22 | bin=`cd "$bin"; pwd`
23 |
24 | . "$bin"/hama-config.sh
25 |
26 | # start bsp daemons
27 | # start zookeeper first to minimize connection errors at startup
28 | "$bin"/hama-daemons.sh --config "${HAMA_CONF_DIR}" start zookeeper
29 | "$bin"/hama-daemon.sh --config $HAMA_CONF_DIR start bspmaster
30 | "$bin"/hama-daemons.sh --config $HAMA_CONF_DIR start groom
31 |
--------------------------------------------------------------------------------
/bin/stop-bspd.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 |
3 | # Licensed to the Apache Software Foundation (ASF) under one or more
4 | # contributor license agreements. See the NOTICE file distributed with
5 | # this work for additional information regarding copyright ownership.
6 | # The ASF licenses this file to You under the Apache License, Version 2.0
7 | # (the "License"); you may not use this file except in compliance with
8 | # the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 |
18 |
19 | # Stop hama map reduce daemons. Run this on master node.
20 |
21 | bin=`dirname "$0"`
22 | bin=`cd "$bin"; pwd`
23 |
24 | . "$bin"/hama-config.sh
25 |
26 | "$bin"/hama-daemon.sh --config $HAMA_CONF_DIR stop bspmaster
27 | "$bin"/hama-daemons.sh --config $HAMA_CONF_DIR stop groom
28 | "$bin"/hama-daemons.sh --config "${HAMA_CONF_DIR}" stop zookeeper
29 |
--------------------------------------------------------------------------------
/bin/zookeepers.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 | #
3 | #/**
4 | # * Copyright 2009 The Apache Software Foundation
5 | # *
6 | # * Licensed to the Apache Software Foundation (ASF) under one
7 | # * or more contributor license agreements. See the NOTICE file
8 | # * distributed with this work for additional information
9 | # * regarding copyright ownership. The ASF licenses this file
10 | # * to you under the Apache License, Version 2.0 (the
11 | # * "License"); you may not use this file except in compliance
12 | # * with the License. You may obtain a copy of the License at
13 | # *
14 | # * http://www.apache.org/licenses/LICENSE-2.0
15 | # *
16 | # * Unless required by applicable law or agreed to in writing, software
17 | # * distributed under the License is distributed on an "AS IS" BASIS,
18 | # * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
19 | # * See the License for the specific language governing permissions and
20 | # * limitations under the License.
21 | # */
22 | #
23 | # Run a shell command on all zookeeper hosts.
24 | #
25 | # Environment Variables
26 | #
27 | # HAMA_CONF_DIR Alternate hama conf dir. Default is ${HAMA_HOME}/conf.
28 | # HAMA_SSH_OPTS Options passed to ssh when running remote commands.
29 | #
30 | # Modelled after $HADOOP_HOME/bin/slaves.sh.
31 |
32 | usage="Usage: zookeepers [--config ] command..."
33 |
34 | # if no args specified, show usage
35 | if [ $# -le 0 ]; then
36 | echo $usage
37 | exit 1
38 | fi
39 |
40 | bin=`dirname "$0"`
41 | bin=`cd "$bin"; pwd`
42 |
43 | . "$bin"/hama-config.sh
44 |
45 | if [ "$HAMA_MANAGES_ZK" = "" ]; then
46 | HAMA_MANAGES_ZK=true
47 | fi
48 |
49 | if [ "$HAMA_MANAGES_ZK" = "true" ]; then
50 | hosts=`"$bin"/hama org.apache.hama.zookeeper.ZKServerTool`
51 | cmd=$"${@// /\\ }"
52 | for zookeeper in $hosts; do
53 | ssh $HAMA_SSH_OPTS $zookeeper $cmd 2>&1 | sed "s/^/$zookeeper: /" &
54 | done
55 | fi
56 |
57 | wait
58 |
--------------------------------------------------------------------------------
/c++/README.txt:
--------------------------------------------------------------------------------
1 | ####################################################################
2 | # Hama Pipes README #
3 | ####################################################################
4 | # Hama Pipes includes the following three examples: #
5 | # - 1) Summation #
6 | # - 2) PiEstimator #
7 | # - 3) MatrixMultiplication #
8 | # in c++/src/main/native/examples #
9 | # Please see c++/src/main/native/examples/README.txt #
10 | ####################################################################
11 |
12 | Please use the following command to compile:
13 |
14 | % g++ -m64 -Ic++/src/main/native/utils/api \
15 | -Ic++/src/main/native/pipes/api \
16 | -Lc++/target/native \
17 | -lhadooputils -lpthread \
18 | PROGRAM.cc \
19 | -o PROGRAM \
20 | -g -Wall -O2
21 |
22 | Attention: The paths have to be adjusted, if you are not operating
23 | in the Hama source folder.
24 |
25 | ####################################################################
26 |
--------------------------------------------------------------------------------
/c++/src/main/native/utils/api/hadoop/Splitter.hh:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | #include
20 | #include
21 |
22 | namespace HadoopUtils {
23 |
24 | class Splitter {
25 | private:
26 | std::vector _tokens;
27 | public:
28 | typedef std::vector::size_type size_type;
29 | public:
30 | Splitter ( const std::string& src, const std::string& delim );
31 |
32 | std::string& operator[] ( size_type i );
33 |
34 | size_type size();
35 |
36 | void reset ( const std::string& src, const std::string& delim );
37 | };
38 | }
39 |
--------------------------------------------------------------------------------
/commons/src/main/java/org/apache/hama/commons/io/FloatArrayWritable.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.commons.io;
19 |
20 | import org.apache.hadoop.io.ArrayWritable;
21 | import org.apache.hadoop.io.FloatWritable;
22 | import org.apache.hadoop.io.Writable;
23 |
24 | public class FloatArrayWritable extends ArrayWritable {
25 | public FloatArrayWritable() {
26 | super(FloatWritable.class);
27 | }
28 |
29 | public String toString() {
30 | Writable[] array = this.get();
31 | StringBuilder s = new StringBuilder();
32 | for (int i = 0; i < array.length; i++) {
33 | s.append(array[i] + " ");
34 | }
35 |
36 | return s.toString();
37 | }
38 | }
39 |
--------------------------------------------------------------------------------
/commons/src/main/java/org/apache/hama/commons/io/TextArrayWritable.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.commons.io;
19 |
20 | import org.apache.hadoop.io.ArrayWritable;
21 | import org.apache.hadoop.io.Text;
22 | import org.apache.hadoop.io.Writable;
23 |
24 | public class TextArrayWritable extends ArrayWritable {
25 |
26 | public TextArrayWritable() {
27 | super(Text.class);
28 | }
29 |
30 | public String toString() {
31 | Writable[] array = this.get();
32 | StringBuilder s = new StringBuilder();
33 | for (int i = 0; i < array.length; i++) {
34 | s.append(array[i] + " ");
35 | }
36 |
37 | return s.toString();
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/commons/src/main/java/org/apache/hama/commons/math/DoubleDoubleFunction.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.commons.math;
19 |
20 | /**
21 | * A double double function takes two arguments. A vector or matrix can apply
22 | * the double function to each element.
23 | *
24 | */
25 | public abstract class DoubleDoubleFunction extends Function {
26 |
27 | /**
28 | * Apply the function to elements to two given arguments.
29 | *
30 | * @param x1
31 | * @param x2
32 | * @return The result based on the calculation on two arguments.
33 | */
34 | public abstract double apply(double x1, double x2);
35 |
36 | /**
37 | * Apply the derivative of this function to two given arguments.
38 | *
39 | * @param x1
40 | * @param x2
41 | * @return The result based on the calculation on two arguments.
42 | */
43 | public abstract double applyDerivative(double x1, double x2);
44 |
45 | }
46 |
--------------------------------------------------------------------------------
/commons/src/main/java/org/apache/hama/commons/math/DoubleDoubleVectorFunction.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.commons.math;
19 |
20 | /**
21 | * A function that can be applied to two double vectors via {@link DoubleVector}
22 | * #apply({@link DoubleVector} v, {@link DoubleDoubleVectorFunction} f);
23 | *
24 | * This class will be replaced by {@link DoubleDoubleFunction}
25 | */
26 | @Deprecated
27 | public interface DoubleDoubleVectorFunction {
28 |
29 | /**
30 | * Calculates the result of the left and right value of two vectors at a given
31 | * index.
32 | */
33 | public double calculate(int index, double left, double right);
34 |
35 | }
36 |
--------------------------------------------------------------------------------
/commons/src/main/java/org/apache/hama/commons/math/DoubleFunction.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.commons.math;
19 |
20 | /**
21 | * A double double function takes two arguments. A vector or matrix can apply
22 | * the double function to each element.
23 | *
24 | */
25 | public abstract class DoubleFunction extends Function {
26 |
27 | /**
28 | * Apply the function to element.
29 | *
30 | * @param value The element that the function apply to.
31 | * @return The result after applying the function.
32 | */
33 | public abstract double apply(double value);
34 |
35 | /**
36 | * Apply the gradient of the function.
37 | *
38 | * @param value
39 | * @return The result after applying the function.
40 | */
41 | public abstract double applyDerivative(double value);
42 |
43 | }
44 |
--------------------------------------------------------------------------------
/commons/src/main/java/org/apache/hama/commons/math/DoubleVectorFunction.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.commons.math;
19 |
20 | /**
21 | * A function that can be applied to a double vector via {@link DoubleVector}
22 | * #apply({@link DoubleVectorFunction} f);
23 | *
24 | * This class will be replaced by {@link DoubleFunction}
25 | */
26 | @Deprecated
27 | public interface DoubleVectorFunction {
28 |
29 | /**
30 | * Calculates the result with a given index and value of a vector.
31 | */
32 | public double calculate(int index, double value);
33 |
34 | }
35 |
--------------------------------------------------------------------------------
/commons/src/main/java/org/apache/hama/commons/math/FloatFloatFunction.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.commons.math;
19 |
20 | public abstract class FloatFloatFunction extends Function {
21 |
22 | /**
23 | * Apply the function to elements to two given arguments.
24 | *
25 | * @param x1
26 | * @param x2
27 | * @return The result based on the calculation on two arguments.
28 | */
29 | public abstract float apply(float x1, float x2);
30 |
31 | /**
32 | * Apply the derivative of this function to two given arguments.
33 | *
34 | * @param x1
35 | * @param x2
36 | * @return The result based on the calculation on two arguments.
37 | */
38 | public abstract float applyDerivative(float x1, float x2);
39 |
40 | }
41 |
--------------------------------------------------------------------------------
/commons/src/main/java/org/apache/hama/commons/math/FloatFunction.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.commons.math;
19 |
20 | public abstract class FloatFunction extends Function {
21 |
22 | /**
23 | * Apply the function to element.
24 | *
25 | * @param value The element that the function apply to.
26 | * @return The result after applying the function.
27 | */
28 | public abstract float apply(float value);
29 |
30 | /**
31 | * Apply the gradient of the function.
32 | *
33 | * @param value
34 | * @return The result after applying the function.
35 | */
36 | public abstract float applyDerivative(float value);
37 |
38 | }
39 |
--------------------------------------------------------------------------------
/commons/src/main/java/org/apache/hama/commons/math/Function.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.commons.math;
19 |
20 | /**
21 | * A generic function.
22 | *
23 | */
24 | public abstract class Function {
25 | /**
26 | * Get the name of the function.
27 | *
28 | * @return The name of the function.
29 | */
30 | final public String getFunctionName() {
31 | return this.getClass().getSimpleName();
32 | }
33 | }
34 |
--------------------------------------------------------------------------------
/commons/src/main/java/org/apache/hama/commons/math/SquareVectorFunction.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.commons.math;
19 |
20 | public class SquareVectorFunction extends DoubleFunction {
21 |
22 | /* (non-Javadoc)
23 | * @see org.apache.hama.commons.math.DoubleFunction#apply(double)
24 | */
25 | @Override
26 | public double apply(double value) {
27 | return value * value;
28 | }
29 |
30 | /* (non-Javadoc)
31 | * @see org.apache.hama.commons.math.DoubleFunction#applyDerivative(double)
32 | */
33 | @Override
34 | public double applyDerivative(double value) {
35 | throw new UnsupportedOperationException();
36 | }
37 |
38 | }
39 |
--------------------------------------------------------------------------------
/commons/src/main/java/org/apache/hama/commons/util/KeyValuePair.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.commons.util;
19 |
20 | /**
21 | * Mutable class for key values.
22 | */
23 | public class KeyValuePair {
24 |
25 | private K key;
26 | private V value;
27 |
28 | public KeyValuePair() {
29 |
30 | }
31 |
32 | public KeyValuePair(K key, V value) {
33 | super();
34 | this.key = key;
35 | this.value = value;
36 | }
37 |
38 | public K getKey() {
39 | return key;
40 | }
41 |
42 | public V getValue() {
43 | return value;
44 | }
45 |
46 | public void setKey(K key) {
47 | this.key = key;
48 | }
49 |
50 | public void setValue(V value) {
51 | this.value = value;
52 | }
53 |
54 | public void clear() {
55 | this.key = null;
56 | this.value = null;
57 | }
58 |
59 | }
60 |
--------------------------------------------------------------------------------
/conf/groomservers:
--------------------------------------------------------------------------------
1 | master.edward.org
2 | slave.edward.org
3 |
--------------------------------------------------------------------------------
/conf/hama-env.sh:
--------------------------------------------------------------------------------
1 | #
2 | #/**
3 | # * Copyright 2007 The Apache Software Foundation
4 | # *
5 | # * Licensed to the Apache Software Foundation (ASF) under one
6 | # * or more contributor license agreements. See the NOTICE file
7 | # * distributed with this work for additional information
8 | # * regarding copyright ownership. The ASF licenses this file
9 | # * to you under the Apache License, Version 2.0 (the
10 | # * "License"); you may not use this file except in compliance
11 | # * with the License. You may obtain a copy of the License at
12 | # *
13 | # * http://www.apache.org/licenses/LICENSE-2.0
14 | # *
15 | # * Unless required by applicable law or agreed to in writing, software
16 | # * distributed under the License is distributed on an "AS IS" BASIS,
17 | # * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
18 | # * See the License for the specific language governing permissions and
19 | # * limitations under the License.
20 | # */
21 |
22 | # Set environment variables here.
23 |
24 | # The java implementation to use. Required.
25 | export JAVA_HOME=/usr/lib/jvm/java-8-oracle
26 |
27 | # Where log files are stored. $HAMA_HOME/logs by default.
28 | # export HAMA_LOG_DIR=${HAMA_HOME}/logs
29 |
30 | # The maximum amount of heap to use, in MB. Default is 1000.
31 | # export HAMA_HEAPSIZE=1000
32 |
33 | # Extra ssh options. Empty by default.
34 | # export HAMA_SSH_OPTS="-o ConnectTimeout=1 -o SendEnv=HAMA_CONF_DIR"
35 |
36 | # Tell Hama whether it should manage it's own instance of Zookeeper or not.
37 | # export HAMA_MANAGES_ZK=true
38 |
--------------------------------------------------------------------------------
/contrib/avro-rpc/src/main/java/org/apache/hama/avro/Sender.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.message;
19 |
20 | import org.apache.hadoop.io.Writable;
21 |
22 | public interface Sender {
23 |
24 | public static final org.apache.avro.Protocol PROTOCOL = org.apache.avro.Protocol
25 | .parse("{\"protocol\":\"Sender\",\"namespace\":\"de.jungblut.avro\",\"types\":[{\"type\":\"record\",\"name\":\"AvroBSPMessageBundle\",\"fields\":[{\"name\":\"data\",\"type\":\"bytes\"}]}],\"messages\":{\"transfer\":{\"request\":[{\"name\":\"messagebundle\",\"type\":\"AvroBSPMessageBundle\"}],\"response\":\"null\"}}}");
26 |
27 | java.lang.Void transfer(AvroBSPMessageBundle messagebundle)
28 | throws org.apache.avro.AvroRemoteException;
29 |
30 | @SuppressWarnings("all")
31 | public interface Callback extends Sender {
32 | public static final org.apache.avro.Protocol PROTOCOL = Sender.PROTOCOL;
33 |
34 | public void transfer(AvroBSPMessageBundle messagebundle,
35 | org.apache.avro.ipc.Callback callback)
36 | throws java.io.IOException;
37 | }
38 | }
39 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/ZooKeeperRunner.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama;
19 |
20 | import org.apache.commons.logging.Log;
21 | import org.apache.commons.logging.LogFactory;
22 | import org.apache.hadoop.conf.Configured;
23 | import org.apache.hadoop.util.Tool;
24 | import org.apache.hadoop.util.ToolRunner;
25 | import org.apache.hama.zookeeper.QuorumPeer;
26 |
27 | /**
28 | * This class starts and runs the ZooKeeperServer.
29 | */
30 | public class ZooKeeperRunner extends Configured implements Tool {
31 |
32 | public static final Log LOG = LogFactory.getLog(ZooKeeperRunner.class);
33 |
34 | @Override
35 | public int run(String[] args) throws Exception {
36 | QuorumPeer.run(new HamaConfiguration());
37 | return 0;
38 | }
39 |
40 | public static void main(String[] args) throws Exception {
41 | int exitCode = ToolRunner.run(new ZooKeeperRunner(), args);
42 | System.exit(exitCode);
43 | }
44 |
45 | }
46 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/BSP.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.io.IOException;
21 |
22 | import org.apache.hadoop.io.Writable;
23 | import org.apache.hama.bsp.sync.SyncException;
24 |
25 | /**
26 | * This class provides an abstract implementation of the {@link BSPInterface}.
27 | */
28 | public abstract class BSP implements
29 | BSPInterface {
30 |
31 | /**
32 | * {@inheritDoc}
33 | */
34 | @Override
35 | public abstract void bsp(BSPPeer peer) throws IOException,
36 | SyncException, InterruptedException;
37 |
38 | /**
39 | * {@inheritDoc}
40 | */
41 | @Override
42 | public void setup(BSPPeer peer) throws IOException,
43 | SyncException, InterruptedException {
44 |
45 | }
46 |
47 | /**
48 | * {@inheritDoc}
49 | */
50 | @Override
51 | public void cleanup(BSPPeer peer) throws IOException {
52 |
53 | }
54 |
55 | }
56 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/BSPMessageBundleInterface.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.util.Iterator;
21 |
22 | import org.apache.hadoop.io.Writable;
23 |
24 | public interface BSPMessageBundleInterface {
25 |
26 | /**
27 | * @return the number of the messages.
28 | */
29 | public int size();
30 |
31 | /**
32 | * Add message to this bundle.
33 | *
34 | * @param message BSPMessage to add.
35 | */
36 | public void addMessage(M message);
37 |
38 | /**
39 | * @return the iterator.
40 | */
41 | public Iterator iterator();
42 |
43 | }
44 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/BSPTaskRunner.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import org.apache.commons.logging.Log;
21 | import org.apache.commons.logging.LogFactory;
22 |
23 | /**
24 | * Base class that runs a task in a separate process.
25 | */
26 | public class BSPTaskRunner extends TaskRunner {
27 |
28 | public static final Log LOG = LogFactory.getLog(BSPTaskRunner.class);
29 |
30 | public BSPTaskRunner(BSPTask bspTask, GroomServer groom, BSPJob conf) {
31 | super(bspTask, groom, conf);
32 | }
33 |
34 | }
35 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/Combiner.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import org.apache.hadoop.io.Writable;
21 |
22 | public abstract class Combiner {
23 |
24 | /**
25 | * Combines messages
26 | *
27 | * @param messages
28 | * @return the combined message
29 | */
30 | public abstract M combine(Iterable messages);
31 |
32 | }
33 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/CommitTaskAction.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.io.DataInput;
21 | import java.io.DataOutput;
22 | import java.io.IOException;
23 |
24 | /**
25 | * Represents a directive from the {@link org.apache.hama.bsp.BSPMaster} to the
26 | * {@link org.apache.hama.bsp.GroomServer} to commit the output of the task.
27 | */
28 | class CommitTaskAction extends GroomServerAction {
29 | private TaskAttemptID taskId;
30 |
31 | public CommitTaskAction() {
32 | super(ActionType.COMMIT_TASK);
33 | taskId = new TaskAttemptID();
34 | }
35 |
36 | public CommitTaskAction(TaskAttemptID taskId) {
37 | super(ActionType.COMMIT_TASK);
38 | this.taskId = taskId;
39 | }
40 |
41 | public TaskAttemptID getTaskID() {
42 | return taskId;
43 | }
44 |
45 | @Override
46 | public void write(DataOutput out) throws IOException {
47 | taskId.write(out);
48 | }
49 |
50 | @Override
51 | public void readFields(DataInput in) throws IOException {
52 | taskId.readFields(in);
53 | }
54 | }
55 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/DirectiveException.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | /**
21 | * A custom exception class for Directive.
22 | */
23 | public class DirectiveException extends RuntimeException {
24 | private static final long serialVersionUID = -8052582046894492822L;
25 |
26 | public DirectiveException() {
27 | super();
28 | }
29 |
30 | public DirectiveException(String message) {
31 | super(message);
32 | }
33 |
34 | public DirectiveException(String message, Throwable t) {
35 | super(message, t);
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/DirectiveHandler.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | /**
21 | * A DirectiveHandler interface.
22 | */
23 | public interface DirectiveHandler {
24 |
25 | /**
26 | * Handle directives on demand.
27 | *
28 | * @param directive to be handled.
29 | */
30 | void handle(Directive directive) throws DirectiveException;
31 | }
32 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/GroomStatusListener.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | /**
21 | * A listener for updates to {@link GroomServerStatus} by groom servers.
22 | */
23 | public interface GroomStatusListener {
24 |
25 | /**
26 | * Invoked when a new groom server has been registered with the {@link BSPMaster}.
27 | *
28 | * @param status The status of the new groom server
29 | */
30 | public abstract void groomServerRegistered(GroomServerStatus status);
31 |
32 | public abstract void taskComplete(GroomServerStatus status, TaskInProgress task);
33 | }
34 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/HashPartitioner.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | /**
21 | * HashPartitioner is partitioning by the hashcode of the key.
22 | *
23 | */
24 | public class HashPartitioner implements Partitioner {
25 |
26 | @Override
27 | public int getPartition(K key, V value, int numTasks) {
28 | return Math.abs(key.hashCode() % numTasks);
29 | }
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/InputFormat.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.io.IOException;
21 |
22 | public interface InputFormat {
23 |
24 | InputSplit[] getSplits(BSPJob job, int numBspTask) throws IOException;
25 |
26 | RecordReader getRecordReader(InputSplit split, BSPJob job)
27 | throws IOException;
28 |
29 | }
30 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/InputSplit.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.io.IOException;
21 |
22 | import org.apache.hadoop.io.Writable;
23 |
24 | public interface InputSplit extends Writable {
25 |
26 | /**
27 | * Get the total number of bytes in the data of the InputSplit.
28 | *
29 | * @return the number of bytes in the input split.
30 | * @throws IOException
31 | */
32 | long getLength() throws IOException;
33 |
34 | /**
35 | * Get the list of hostnames where the input split is located.
36 | *
37 | * @return list of hostnames where data of the InputSplit is
38 | * located as an array of Strings.
39 | * @throws IOException
40 | */
41 | String[] getLocations() throws IOException;
42 | }
43 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/JobChangeEvent.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | /**
21 | * {@link JobChangeEvent} is used to capture state changes in a job. A job can
22 | * change its state w.r.t priority, progress, run-state etc.
23 | */
24 | abstract class JobChangeEvent {
25 | private JobInProgress jip;
26 |
27 | JobChangeEvent(JobInProgress jip) {
28 | this.jip = jip;
29 | }
30 |
31 | /**
32 | * Get the job object for which the change is reported
33 | */
34 | JobInProgress getJobInProgress() {
35 | return jip;
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/KeyValueTextInputFormat.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.hama.bsp;
20 |
21 | import java.io.IOException;
22 |
23 | import org.apache.hadoop.io.Text;
24 |
25 | /**
26 | * An {@link InputFormat} for plain text files. Files are broken into lines.
27 | * Either linefeed or carriage-return are used to signal end of line. Each line
28 | * is divided into key and value parts by a separator byte. If no such a byte
29 | * exists, the key will be the entire line and value will be empty.
30 | */
31 | public class KeyValueTextInputFormat extends FileInputFormat {
32 |
33 | @Override
34 | public RecordReader getRecordReader(InputSplit genericSplit,
35 | BSPJob job) throws IOException {
36 | return new KeyValueLineRecordReader(job.conf, (FileSplit) genericSplit);
37 | }
38 |
39 | }
40 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/KillJobAction.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.io.DataInput;
21 | import java.io.DataOutput;
22 | import java.io.IOException;
23 |
24 | import org.apache.hadoop.io.Text;
25 |
26 | /**
27 | * Represents a directive from the {@link org.apache.hama.bsp.BSPMaster} to the
28 | * {@link org.apache.hama.bsp.GroomServer} to kill the task of a job and cleanup
29 | * resources.
30 | */
31 | class KillJobAction extends GroomServerAction {
32 | String jobId;
33 |
34 | public KillJobAction() {
35 | super(ActionType.KILL_JOB);
36 | jobId = "";
37 | }
38 |
39 | public KillJobAction(String killJobId) {
40 | super(ActionType.KILL_JOB);
41 | this.jobId = killJobId;
42 | }
43 |
44 | public String getJobID() {
45 | return jobId;
46 | }
47 |
48 | @Override
49 | public void write(DataOutput out) throws IOException {
50 | Text.writeString(out, jobId);
51 | }
52 |
53 | @Override
54 | public void readFields(DataInput in) throws IOException {
55 | jobId = Text.readString(in);
56 | }
57 |
58 | }
59 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/KillTaskAction.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.io.DataInput;
21 | import java.io.DataOutput;
22 | import java.io.IOException;
23 |
24 | /**
25 | * Represents a directive from the {@link org.apache.hama.bsp.BSPMaster} to the
26 | * {@link org.apache.hama.bsp.GroomServer} to kill a task.
27 | */
28 | class KillTaskAction extends GroomServerAction {
29 | TaskAttemptID taskId;
30 |
31 | public KillTaskAction() {
32 | super(ActionType.KILL_TASK);
33 | taskId = new TaskAttemptID();
34 | }
35 |
36 | public KillTaskAction(TaskAttemptID killTaskId) {
37 | super(ActionType.KILL_TASK);
38 | this.taskId = killTaskId;
39 | }
40 |
41 | public TaskAttemptID getTaskID() {
42 | return taskId;
43 | }
44 |
45 | @Override
46 | public void write(DataOutput out) throws IOException {
47 | taskId.write(out);
48 | }
49 |
50 | @Override
51 | public void readFields(DataInput in) throws IOException {
52 | taskId.readFields(in);
53 | }
54 | }
55 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/LaunchTaskAction.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.io.DataInput;
21 | import java.io.DataOutput;
22 | import java.io.IOException;
23 |
24 | /**
25 | * Represents a directive from the {@link org.apache.hama.bsp.BSPMaster} to the
26 | * {@link org.apache.hama.bsp.GroomServer} to launch a recovery task.
27 | */
28 | class LaunchTaskAction extends GroomServerAction {
29 | private Task task;
30 |
31 | public LaunchTaskAction() {
32 | super(ActionType.LAUNCH_TASK);
33 | }
34 |
35 | public LaunchTaskAction(Task task) {
36 | super(ActionType.LAUNCH_TASK);
37 | this.task = task;
38 | }
39 |
40 | public Task getTask() {
41 | return task;
42 | }
43 |
44 | @Override
45 | public void write(DataOutput out) throws IOException {
46 | task.write(out);
47 | }
48 |
49 | @Override
50 | public void readFields(DataInput in) throws IOException {
51 | task = new BSPTask();
52 | task.readFields(in);
53 | }
54 |
55 | }
56 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/Messagable.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | /**
21 | * A interface for BSP message class.
22 | */
23 | public interface Messagable {
24 |
25 | }
26 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/MonitorManager.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import org.apache.hama.monitor.fd.Supervisor;
21 |
22 | public interface MonitorManager {
23 |
24 | /**
25 | * Provide interface accessing to Supervisor.
26 | */
27 | Supervisor supervisor();
28 |
29 | }
30 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/NonSplitSequenceFileInputFormat.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import org.apache.hadoop.fs.Path;
21 |
22 | /**
23 | * Only for input partitioning job.
24 | */
25 | public class NonSplitSequenceFileInputFormat extends
26 | SequenceFileInputFormat {
27 |
28 | @Override
29 | protected boolean isSplitable(BSPJob job, Path path) {
30 | return false;
31 | }
32 | }
33 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/NullOutputFormat.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import org.apache.hadoop.fs.FileSystem;
21 |
22 | public class NullOutputFormat implements OutputFormat {
23 |
24 | @Override
25 | public RecordWriter getRecordWriter(FileSystem ignored, BSPJob job,
26 | String name) {
27 | return new RecordWriter() {
28 | @Override
29 | public void write(K key, V value) {
30 | }
31 |
32 | @Override
33 | public void close() {
34 | }
35 | };
36 | }
37 |
38 | @Override
39 | public void checkOutputSpecs(FileSystem ignored, BSPJob job) {
40 | }
41 | }
42 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/OutputCollector.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.io.IOException;
21 |
22 | public interface OutputCollector {
23 |
24 | /**
25 | * Adds a key/value pair to the output.
26 | *
27 | * @param key the key to collect.
28 | * @param value to value to collect.
29 | * @throws IOException
30 | */
31 | void collect(K key, V value) throws IOException;
32 | }
33 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/Partitioner.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | /**
21 | * Partitioning interface which is used to spread key value pairs to a specific
22 | * partition.
23 | *
24 | * @param
25 | * @param
26 | */
27 | public interface Partitioner {
28 |
29 | /**
30 | * Partitions a specific key value mapping to a bucket.
31 | *
32 | * @param key
33 | * @param value
34 | * @param numTasks
35 | * @return a number between 0 and numTasks (exclusive) that tells which
36 | * partition it belongs to.
37 | */
38 | public int getPartition(K key, V value, int numTasks);
39 |
40 | }
41 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/Queue.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.util.Collection;
21 |
22 | /**
23 | * Job Queue interface.
24 | *
25 | * @param
26 | */
27 | public interface Queue {
28 |
29 | /**
30 | * The queue name.
31 | *
32 | * @return the name of current queue.
33 | */
34 | String getName();
35 |
36 | /**
37 | * Add a job to a queue.
38 | *
39 | * @param job to be added to the queue.
40 | */
41 | void addJob(T job);
42 |
43 | /**
44 | * Remove a job from the queue.
45 | *
46 | * @param job to be removed from the queue.
47 | */
48 | void removeJob(T job);
49 |
50 | /**
51 | * Get a job
52 | *
53 | * @return job that is removed from the queue.
54 | */
55 | T removeJob();
56 |
57 | /**
58 | * Return all data stored in this queue.
59 | *
60 | * @return Collection of jobs.
61 | */
62 | public Collection jobs();
63 |
64 | }
65 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/RecordWriter.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.io.IOException;
21 |
22 | public interface RecordWriter {
23 | /**
24 | * Writes a key/value pair.
25 | *
26 | * @param key the key to write.
27 | * @param value the value to write.
28 | * @throws IOException
29 | */
30 | void write(K key, V value) throws IOException;
31 |
32 | /**
33 | * Close this RecordWriter to future operations.
34 | *
35 | * @throws IOException
36 | */
37 | void close() throws IOException;
38 | }
39 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/ReinitGroomAction.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.io.DataInput;
21 | import java.io.DataOutput;
22 | import java.io.IOException;
23 |
24 | /**
25 | * Represents a directive from the {@link org.apache.hama.bsp.BSPMaster} to the
26 | * {@link org.apache.hama.bsp.GroomServer} to reinitialize itself.
27 | */
28 | class ReinitGroomAction extends GroomServerAction {
29 |
30 | public ReinitGroomAction() {
31 | super(ActionType.REINIT_GROOM);
32 | }
33 |
34 | @Override
35 | public void write(DataOutput out) throws IOException {
36 | }
37 |
38 | @Override
39 | public void readFields(DataInput in) throws IOException {
40 | }
41 |
42 | }
43 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/Schedulable.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.io.IOException;
21 |
22 | /**
23 | * This is the class that schedules commands to GroomServer(s)
24 | */
25 | public interface Schedulable {
26 |
27 | /**
28 | * Schedule job immediately.
29 | *
30 | * @param job to be scheduled.
31 | * @throws IOException
32 | */
33 | void schedule(JobInProgress job)
34 | throws IOException;
35 | }
36 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/SequenceFileOutputFormat.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2007 The Apache Software Foundation
3 | *
4 | * Licensed to the Apache Software Foundation (ASF) under one
5 | * or more contributor license agreements. See the NOTICE file
6 | * distributed with this work for additional information
7 | * regarding copyright ownership. The ASF licenses this file
8 | * to you under the Apache License, Version 2.0 (the
9 | * "License"); you may not use this file except in compliance
10 | * with the License. You may obtain a copy of the License at
11 | *
12 | * http://www.apache.org/licenses/LICENSE-2.0
13 | *
14 | * Unless required by applicable law or agreed to in writing, software
15 | * distributed under the License is distributed on an "AS IS" BASIS,
16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17 | * See the License for the specific language governing permissions and
18 | * limitations under the License.
19 | */
20 | package org.apache.hama.bsp;
21 |
22 | import java.io.IOException;
23 |
24 | import org.apache.hadoop.fs.FileSystem;
25 | import org.apache.hadoop.io.Writable;
26 |
27 | public class SequenceFileOutputFormat
28 | extends FileOutputFormat {
29 |
30 | @Override
31 | public RecordWriter getRecordWriter(FileSystem fs, BSPJob job,
32 | String name) throws IOException {
33 | try {
34 | return new SequenceFileRecordWriter(fs, job, name);
35 | } catch (ClassNotFoundException e) {
36 | e.printStackTrace();
37 | }
38 | return null;
39 | }
40 | }
41 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/Superstep.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.io.IOException;
21 |
22 | import org.apache.hadoop.io.Writable;
23 |
24 | public abstract class Superstep {
25 |
26 | /**
27 | * Setup this superstep, is called once before compute().
28 | */
29 | protected void setup(BSPPeer peer) {
30 |
31 | }
32 |
33 | /**
34 | * Cleanup this superstep, is called once after compute().
35 | */
36 | protected void cleanup(BSPPeer peer) {
37 |
38 | }
39 |
40 | /**
41 | * Main computation phase.
42 | */
43 | protected abstract void compute(
44 | BSPPeer peer)
45 | throws IOException;
46 |
47 | /**
48 | * @return true to halt the computation
49 | */
50 | protected boolean haltComputation(
51 | BSPPeer peer) {
52 | return false;
53 | }
54 |
55 | }
56 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/TextInputFormat.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp;
19 |
20 | import java.io.IOException;
21 |
22 | import org.apache.hadoop.fs.Path;
23 | import org.apache.hadoop.io.LongWritable;
24 | import org.apache.hadoop.io.Text;
25 | import org.apache.hadoop.io.compress.CompressionCodec;
26 | import org.apache.hadoop.io.compress.CompressionCodecFactory;
27 |
28 | public class TextInputFormat extends FileInputFormat {
29 |
30 | @Override
31 | public RecordReader getRecordReader(InputSplit split,
32 | BSPJob job) throws IOException {
33 | return new LineRecordReader(job.getConfiguration(), (FileSplit) split);
34 | }
35 |
36 | @Override
37 | protected boolean isSplitable(BSPJob job, Path path) {
38 | CompressionCodec codec = new CompressionCodecFactory(job.getConfiguration())
39 | .getCodec(path);
40 | return codec == null;
41 | }
42 |
43 | }
44 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/join/ComposableInputFormat.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.join;
19 |
20 | import java.io.IOException;
21 |
22 | import org.apache.hadoop.io.Writable;
23 | import org.apache.hadoop.io.WritableComparable;
24 | import org.apache.hama.bsp.BSPJob;
25 | import org.apache.hama.bsp.InputFormat;
26 | import org.apache.hama.bsp.InputSplit;
27 |
28 | /**
29 | * Refinement of InputFormat requiring implementors to provide
30 | * ComposableRecordReader instead of RecordReader.
31 | */
32 | public interface ComposableInputFormat
33 | extends InputFormat {
34 |
35 | ComposableRecordReader getRecordReader(InputSplit split, BSPJob job)
36 | throws IOException;
37 | }
38 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/join/InnerJoinRecordReader.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.join;
19 |
20 | import java.io.IOException;
21 |
22 | import org.apache.hadoop.io.WritableComparable;
23 | import org.apache.hadoop.io.WritableComparator;
24 | import org.apache.hama.bsp.BSPJob;
25 |
26 | /**
27 | * Full inner join.
28 | */
29 | public class InnerJoinRecordReader extends
30 | JoinRecordReader {
31 |
32 | InnerJoinRecordReader(int id, BSPJob job, int capacity,
33 | Class extends WritableComparator> cmpcl) throws IOException {
34 | super(id, job, capacity, cmpcl);
35 | }
36 |
37 | /**
38 | * Return true iff the tuple is full (all data sources contain this key).
39 | */
40 | protected boolean combine(Object[] srcs, TupleWritable dst) {
41 | assert srcs.length == dst.size();
42 | for (int i = 0; i < srcs.length; ++i) {
43 | if (!dst.has(i)) {
44 | return false;
45 | }
46 | }
47 | return true;
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/join/OuterJoinRecordReader.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.join;
19 |
20 | import java.io.IOException;
21 |
22 | import org.apache.hadoop.io.WritableComparable;
23 | import org.apache.hadoop.io.WritableComparator;
24 | import org.apache.hama.bsp.BSPJob;
25 |
26 | /**
27 | * Full outer join.
28 | */
29 | public class OuterJoinRecordReader extends
30 | JoinRecordReader {
31 |
32 | OuterJoinRecordReader(int id, BSPJob job, int capacity,
33 | Class extends WritableComparator> cmpcl) throws IOException {
34 | super(id, job, capacity, cmpcl);
35 | }
36 |
37 | /**
38 | * Emit everything from the collector.
39 | */
40 | protected boolean combine(Object[] srcs, TupleWritable dst) {
41 | assert srcs.length == dst.size();
42 | return true;
43 | }
44 | }
45 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/message/HamaMessageManager.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.message;
19 |
20 | import java.io.IOException;
21 |
22 | import org.apache.hadoop.io.Writable;
23 | import org.apache.hama.bsp.BSPMessageBundle;
24 | import org.apache.hama.ipc.HamaRPCProtocolVersion;
25 |
26 | /**
27 | * Hama RPC Interface for messaging.
28 | *
29 | */
30 | public interface HamaMessageManager extends
31 | HamaRPCProtocolVersion {
32 |
33 | /**
34 | * This method puts a message for the next iteration. Accessed concurrently
35 | * from protocol, this must be synchronized internal.
36 | *
37 | * @param msg
38 | */
39 | public void put(M msg) throws IOException;
40 |
41 | /**
42 | * This method puts a messagebundle for the next iteration. Accessed
43 | * concurrently from protocol, this must be synchronized internal.
44 | *
45 | * @param messages
46 | */
47 | public void put(BSPMessageBundle messages) throws IOException;
48 |
49 | public void put(byte[] compressedBundle) throws IOException;
50 |
51 | }
52 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/message/MessageManagerFactory.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.message;
19 |
20 | import org.apache.hadoop.conf.Configuration;
21 | import org.apache.hadoop.io.Writable;
22 | import org.apache.hadoop.util.ReflectionUtils;
23 |
24 | public class MessageManagerFactory {
25 | public static final String MESSAGE_MANAGER_CLASS = "hama.messenger.class";
26 |
27 | /**
28 | * Returns a messenger via reflection based on what was configured.
29 | *
30 | * @param conf
31 | * @return a messenger that was configured.
32 | */
33 | @SuppressWarnings("unchecked")
34 | public static MessageManager getMessageManager(
35 | Configuration conf) throws ClassNotFoundException {
36 | return (MessageManager) ReflectionUtils.newInstance(conf
37 | .getClassByName(conf.get(MESSAGE_MANAGER_CLASS,
38 | org.apache.hama.bsp.message.HamaMessageManagerImpl.class
39 | .getCanonicalName())), conf);
40 | }
41 |
42 | }
43 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/message/OutgoingMessageManager.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.message;
19 |
20 | import java.net.InetSocketAddress;
21 | import java.util.Iterator;
22 | import java.util.Map.Entry;
23 |
24 | import org.apache.hadoop.io.Writable;
25 | import org.apache.hama.HamaConfiguration;
26 | import org.apache.hama.bsp.BSPMessageBundle;
27 |
28 | public interface OutgoingMessageManager {
29 |
30 | public void init(HamaConfiguration conf);
31 |
32 | public void addMessage(String peerName, M msg);
33 |
34 | public void clear();
35 |
36 | public Iterator>> getBundleIterator();
37 |
38 | }
39 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/message/compress/BSPMessageCompressor.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.message.compress;
19 |
20 | import org.apache.commons.logging.Log;
21 | import org.apache.commons.logging.LogFactory;
22 | import org.apache.hadoop.io.Writable;
23 |
24 | /**
25 | * Provides utilities for compressing and decompressing byte array.
26 | *
27 | */
28 | public abstract class BSPMessageCompressor {
29 |
30 | public static final Log LOG = LogFactory.getLog(BSPMessageCompressor.class);
31 |
32 | public abstract byte[] compress(byte[] bytes);
33 |
34 | public abstract byte[] decompress(byte[] compressedBytes);
35 | }
36 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/message/compress/package.html:
--------------------------------------------------------------------------------
1 |
2 |
3 |
19 |
20 |
21 | Message compressor.
22 |
23 |
24 |
25 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/message/package.html:
--------------------------------------------------------------------------------
1 |
2 |
3 |
19 |
20 |
21 | Contains the implementation of message transfer.
22 |
23 |
24 |
25 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/message/queue/SynchronizedQueue.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.message.queue;
19 |
20 | import org.apache.hadoop.io.Writable;
21 |
22 | /**
23 | * Synchronized Queue interface. Can be used to implement better synchronized
24 | * datastructures.
25 | */
26 | public interface SynchronizedQueue extends MessageQueue {
27 |
28 | public abstract MessageQueue getMessageQueue();
29 |
30 | public abstract void prepareRead();
31 |
32 | }
33 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/message/queue/package.html:
--------------------------------------------------------------------------------
1 |
2 |
3 |
19 |
20 |
21 | Contains all queue implementations.
22 |
23 |
24 |
25 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/package.html:
--------------------------------------------------------------------------------
1 |
2 |
3 |
19 |
20 |
21 | BSP computing framework.
22 |
23 |
24 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/sync/BSPMasterSyncClient.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.sync;
19 |
20 | import org.apache.hama.HamaConfiguration;
21 |
22 | public abstract class BSPMasterSyncClient implements MasterSyncClient {
23 |
24 | /**
25 | * Initialize the Synchronization client.
26 | *
27 | * @param conf The configuration parameters to initialize the client.
28 | */
29 | public abstract void init(HamaConfiguration conf);
30 |
31 | /**
32 | * Clears all information stored.
33 | */
34 | public abstract void clear();
35 |
36 | /**
37 | * Register a newly added job
38 | *
39 | * @param string
40 | */
41 | public abstract void registerJob(String string);
42 |
43 | /**
44 | * Deregister the job from the system.
45 | *
46 | * @param string
47 | */
48 | public abstract void deregisterJob(String string);
49 |
50 | /**
51 | * Closes the client.
52 | */
53 | public abstract void close();
54 |
55 | }
56 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/sync/MasterSyncClient.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.sync;
19 |
20 | import org.apache.hama.HamaConfiguration;
21 |
22 | /**
23 | * MasterSyncClient defines the behavior that BSPMaster should follow to perform
24 | * different required globally synchronized state changes.
25 | *
26 | */
27 | public interface MasterSyncClient extends SyncClient {
28 |
29 | /**
30 | * Initialize the Synchronization client.
31 | *
32 | * @param conf The configuration parameters to initialize the client.
33 | */
34 | public void init(HamaConfiguration conf);
35 |
36 | /**
37 | * Clears all information stored.
38 | */
39 | public void clear();
40 |
41 | /**
42 | * Register a newly added job
43 | *
44 | * @param string
45 | */
46 | public void registerJob(String string);
47 |
48 | /**
49 | * Deregister the job from the system.
50 | *
51 | * @param string
52 | */
53 | public void deregisterJob(String string);
54 |
55 | }
56 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/sync/SyncEvent.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.sync;
19 |
20 | /**
21 | * A distributed global synchronization event.
22 | */
23 | public interface SyncEvent {
24 |
25 | /**
26 | * Returns the event identifier in the scheme of events defined for the global
27 | * synchronization service.
28 | *
29 | * @return the event identifier
30 | */
31 | public int getEventId();
32 |
33 | }
34 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/sync/SyncEventListener.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.sync;
19 |
20 | /**
21 | * This class is used to define a listener to the synchronized global event.
22 | *
23 | */
24 | public abstract class SyncEventListener {
25 |
26 | /**
27 | * Every event is identified by an event identifier. You can refer to
28 | * SyncEvent class.
29 | *
30 | * @param eventId The event identification code.
31 | */
32 | public abstract void handleEvent(int eventId);
33 |
34 | }
35 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/sync/SyncException.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.sync;
19 |
20 | public class SyncException extends Exception {
21 | /**
22 | *
23 | */
24 | private static final long serialVersionUID = 1L;
25 |
26 | String info;
27 |
28 | public SyncException(String info) {
29 | super(); // call superclass constructor
30 | this.info = info;
31 | }
32 |
33 | public String getError() {
34 | return info;
35 | }
36 |
37 | }
38 |
--------------------------------------------------------------------------------
/core/src/main/java/org/apache/hama/bsp/sync/SyncServerRunner.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.hama.bsp.sync;
19 |
20 | import java.util.concurrent.Callable;
21 |
22 | import org.apache.hadoop.conf.Configuration;
23 |
24 | /**
25 | * Thread runner for a sync server.
26 | *
27 | */
28 | public class SyncServerRunner implements Callable