├── 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 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 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 { 29 | 30 | private SyncServer syncServer; 31 | 32 | // use the SyncServiceFactory to obtain a new instance. 33 | SyncServerRunner(Configuration conf) { 34 | try { 35 | syncServer = SyncServiceFactory.getSyncServer(conf); 36 | } catch (ClassNotFoundException e) { 37 | e.printStackTrace(); 38 | } 39 | } 40 | 41 | public Configuration init(Configuration conf) throws Exception { 42 | return syncServer.init(conf); 43 | } 44 | 45 | public void stop() { 46 | syncServer.stopServer(); 47 | } 48 | 49 | @Override 50 | public Object call() throws Exception { 51 | syncServer.start(); 52 | return null; 53 | } 54 | 55 | @Override 56 | protected void finalize() throws Throwable { 57 | syncServer.stopServer(); 58 | } 59 | 60 | } 61 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/bsp/sync/package.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 19 | 20 | 21 | Contains the base implementation of a barrier synchronization agent. 22 | 23 | 24 | 25 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/bsp/taskallocation/BSPResource.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.taskallocation; 19 | 20 | import org.apache.hama.bsp.TaskInProgress; 21 | 22 | /** 23 | * BSPResource defines a resource entity that would be used as a factor 24 | * for allocating tasks on groom-servers. 25 | */ 26 | public abstract class BSPResource { 27 | 28 | /** 29 | * Returns the list of grooms on which the current resource is available or 30 | * local or is best chosen for the task. 31 | * 32 | * @param tip The TaskInProgress representing the task to 33 | * schedule. 34 | * @return The list of groomserver host names. 35 | */ 36 | public abstract String[] getGrooms(TaskInProgress tip); 37 | 38 | } 39 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/bsp/taskallocation/RawSplitResource.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.taskallocation; 19 | 20 | import org.apache.hama.bsp.BSPJobClient.RawSplit; 21 | import org.apache.hama.bsp.TaskInProgress; 22 | 23 | /** 24 | * RawSplitResource defines the data block resource that could be 25 | * used to find which groom to schedule for data-locality. 26 | */ 27 | public class RawSplitResource extends BSPResource { 28 | 29 | private RawSplit split; 30 | 31 | public RawSplitResource() { 32 | 33 | } 34 | 35 | /** 36 | * Initialize the resource with data block split information. 37 | * 38 | * @param split The data-split provided by BSPJobClient 39 | */ 40 | public RawSplitResource(RawSplit split) { 41 | this.split = split; 42 | } 43 | 44 | @Override 45 | public String[] getGrooms(TaskInProgress tip) { 46 | return split.getLocations(); 47 | } 48 | 49 | } 50 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/http/package.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 19 | 20 | 21 | Contains the administrative web interfaces. 22 | 23 | 24 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/ipc/GroomProtocol.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.ipc; 19 | 20 | import java.io.IOException; 21 | 22 | import org.apache.hama.bsp.Directive; 23 | 24 | /** 25 | * A protocol for BSPMaster talks to GroomServer. 26 | */ 27 | public interface GroomProtocol extends HamaRPCProtocolVersion { 28 | 29 | /** 30 | * Instruct GroomServer performing tasks. 31 | * 32 | * @param directive instructs a GroomServer performing necessary execution. 33 | * @throws IOException 34 | */ 35 | void dispatch(Directive directive) throws IOException; 36 | 37 | } 38 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/ipc/HamaRPCProtocolVersion.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.ipc; 19 | 20 | /** 21 | * There is one version id for all the RPC interfaces. If any interface is 22 | * changed, the versionID must be changed here. 23 | */ 24 | public interface HamaRPCProtocolVersion extends VersionedProtocol { 25 | public static final long versionID = 1L; 26 | } 27 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/ipc/RetryPolicy.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.ipc; 19 | 20 | /** 21 | *

22 | * Specifies a policy for retrying method failures. 23 | * Implementations of this interface should be immutable. 24 | *

25 | */ 26 | public interface RetryPolicy { 27 | /** 28 | *

29 | * Determines whether the framework should retry a 30 | * method for the given exception, and the number 31 | * of retries that have been made for that operation 32 | * so far. 33 | *

34 | * @param e The exception that caused the method to fail. 35 | * @param retries The number of times the method has been retried. 36 | * @return true if the method should be retried, 37 | * false if the method should not be retried 38 | * but shouldn't fail with an exception (only for void methods). 39 | * @throws Exception The re-thrown exception e indicating 40 | * that the method failed and should not be retried further. 41 | */ 42 | public boolean shouldRetry(Exception e, int retries) throws Exception; 43 | } 44 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/ipc/Status.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.ipc; 19 | 20 | /** 21 | * Status of a Hadoop IPC call. 22 | */ 23 | enum Status { 24 | SUCCESS (0), 25 | ERROR (1), 26 | FATAL (-1); 27 | 28 | int state; 29 | private Status(int state) { 30 | this.state = state; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/ipc/VersionedProtocol.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.ipc; 20 | 21 | import java.io.IOException; 22 | 23 | /** 24 | * Superclass of all protocols that use Hadoop RPC. 25 | * Subclasses of this interface are also supposed to have 26 | * a static final long versionID field. 27 | */ 28 | public interface VersionedProtocol { 29 | 30 | /** 31 | * Return protocol version corresponding to protocol interface. 32 | * @param protocol The classname of the protocol interface 33 | * @param clientVersion The version of the protocol that the client speaks 34 | * @return the version that the server will speak 35 | */ 36 | public long getProtocolVersion(String protocol, 37 | long clientVersion) throws IOException; 38 | } 39 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/ipc/package.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 19 | 20 | 21 | Tools to help define network clients and servers. 22 | 23 | 24 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/monitor/MonitorListener.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.monitor; 19 | 20 | import org.apache.hama.monitor.Monitor.Result; 21 | 22 | /** 23 | * MonitorListener passes the result for notification. 24 | */ 25 | public interface MonitorListener { 26 | 27 | /** 28 | * When an event is triggered, the task passes the result to notify the 29 | * monitor. 30 | */ 31 | void notify(Result result); 32 | 33 | } 34 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/monitor/fd/NodeEventListener.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.monitor.fd; 19 | 20 | /** 21 | * Notify when an event happens. 22 | */ 23 | public interface NodeEventListener { 24 | 25 | /** 26 | * Notify the node status. 27 | * 28 | * @param status status of the groom server. 29 | * @param host name of the groom server. 30 | */ 31 | void notify(NodeStatus status, String host); 32 | 33 | /** 34 | * The status that the listener is interested in. 35 | * 36 | * @return the status the listener has interest. 37 | */ 38 | NodeStatus[] interest(); 39 | 40 | /** 41 | * This listener's name. 42 | */ 43 | String name(); 44 | 45 | } 46 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/monitor/fd/NodeStatus.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.monitor.fd; 19 | 20 | public enum NodeStatus { 21 | Alive, Dead 22 | } 23 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/monitor/fd/Sensor.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.monitor.fd; 20 | 21 | import java.io.IOException; 22 | 23 | /** 24 | * Failure detector client, sending heartbeat to supervisor. 25 | */ 26 | public interface Sensor { 27 | 28 | /** 29 | * The heartbeat function, signifying its existence. 30 | */ 31 | void heartbeat() throws IOException; 32 | 33 | /** 34 | * Start sensor. 35 | */ 36 | void start(); 37 | 38 | /** 39 | * Stop sensor. 40 | */ 41 | void stop(); 42 | 43 | } 44 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/monitor/fd/Supervisor.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.monitor.fd; 20 | 21 | /** 22 | * A failure detector component. It is responsible for receiving the heartbeat 23 | * and output suspicion level for Interpreter. 24 | */ 25 | public interface Supervisor { 26 | 27 | /** 28 | * Receive notification if a node fails. 29 | * 30 | * @param listener will be called if a node fails. 31 | */ 32 | void register(NodeEventListener listener); 33 | 34 | /** 35 | * Start supervisor. 36 | */ 37 | void start(); 38 | 39 | /** 40 | * Shutdown supervisor. 41 | */ 42 | void stop(); 43 | 44 | } 45 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/package.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 19 | 20 | 21 | Hama base package. 22 | 23 | 24 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/pipes/protocol/MessageType.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.pipes.protocol; 19 | 20 | /** 21 | * The integer codes to represent the different messages. These must match the 22 | * C++ codes or massive confusion will result. 23 | */ 24 | public enum MessageType { 25 | START(0), SET_BSPJOB_CONF(1), SET_INPUT_TYPES(2), RUN_SETUP(3), RUN_BSP(4), 26 | RUN_CLEANUP(5), READ_KEYVALUE(6), WRITE_KEYVALUE(7), GET_MSG(8), 27 | GET_MSG_COUNT(9), SEND_MSG(10), SYNC(11), GET_ALL_PEERNAME(12), 28 | GET_PEERNAME(13), GET_PEER_INDEX(14), GET_PEER_COUNT(15), 29 | GET_SUPERSTEP_COUNT(16), 30 | REOPEN_INPUT(17), CLEAR(18), CLOSE(19), ABORT(20), DONE(21), TASK_DONE(22), 31 | REGISTER_COUNTER(23), INCREMENT_COUNTER(24), SEQFILE_OPEN(25), 32 | SEQFILE_READNEXT(26), SEQFILE_APPEND(27), SEQFILE_CLOSE(28), 33 | PARTITION_REQUEST(29), PARTITION_RESPONSE(30), LOG(31), END_OF_DATA(32); 34 | 35 | final int code; 36 | 37 | MessageType(int code) { 38 | this.code = code; 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/util/ByteUtils.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.util; 19 | 20 | /** 21 | * Utilities class for byte operations and constants 22 | */ 23 | public class ByteUtils { 24 | /** Bytes used in a boolean */ 25 | public static final int SIZE_OF_BOOLEAN = 1; 26 | /** Bytes used in a byte */ 27 | public static final int SIZE_OF_BYTE = 1; 28 | /** Bytes used in a char */ 29 | public static final int SIZE_OF_CHAR = Character.SIZE / Byte.SIZE; 30 | /** Bytes used in a short */ 31 | public static final int SIZE_OF_SHORT = Short.SIZE / Byte.SIZE; 32 | /** Bytes used in an int */ 33 | public static final int SIZE_OF_INT = Integer.SIZE / Byte.SIZE; 34 | /** Bytes used in a long */ 35 | public static final int SIZE_OF_LONG = Long.SIZE / Byte.SIZE; 36 | /** Bytes used in a float */ 37 | public static final int SIZE_OF_FLOAT = Float.SIZE / Byte.SIZE; 38 | /** Bytes used in a double */ 39 | public static final int SIZE_OF_DOUBLE = Double.SIZE / Byte.SIZE; 40 | } 41 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/util/DistCacheUtils.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.util; 19 | 20 | import org.apache.hadoop.conf.Configuration; 21 | 22 | public class DistCacheUtils { 23 | 24 | private static final String CACHE_LOCALFILES = "mapred.cache.localFiles"; 25 | 26 | public static void addLocalFiles(Configuration conf, String str) { 27 | String files = conf.get(CACHE_LOCALFILES); 28 | conf.set(CACHE_LOCALFILES, files == null ? str 29 | : files + "," + str); 30 | } 31 | 32 | public static void setLocalFiles(Configuration conf, String str) { 33 | conf.set(CACHE_LOCALFILES, str); 34 | } 35 | 36 | } 37 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/util/package.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 19 | 20 | 21 | Common utilities. 22 | 23 | 24 | -------------------------------------------------------------------------------- /core/src/main/java/org/apache/hama/zookeeper/package.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 19 | 20 | 21 | Tools to start the ZooKeeper server and quorum peers. 22 | 23 | 24 | -------------------------------------------------------------------------------- /core/src/main/resources/webapp/bspmaster/index.html: -------------------------------------------------------------------------------- 1 | 17 | 18 | 19 | 20 | 21 | Hama Administration 22 | 23 | 24 | 25 | 26 | 27 |

Hama Administration

28 | 29 | 34 | 35 | 36 | 37 | 38 | -------------------------------------------------------------------------------- /core/src/main/resources/webapp/bspmaster/machines.jsp: -------------------------------------------------------------------------------- 1 | 17 | <%@ page contentType="text/html; charset=UTF-8" import="javax.servlet.*" 18 | import="javax.servlet.http.*" import="java.io.*" import="java.util.*" 19 | import="java.text.DecimalFormat" import="org.apache.hama.bsp.*" 20 | import="org.apache.hama.util.*"%> 21 | <%!private static final long serialVersionUID = 1L;%> 22 | <% 23 | BSPMaster tracker = (BSPMaster) application 24 | .getAttribute("bsp.master"); 25 | ClusterStatus status = tracker.getClusterStatus(true); 26 | String trackerName = tracker.getBSPMasterName(); 27 | String type = request.getParameter("type"); 28 | %> 29 | 30 | 31 | 32 | <%=trackerName%> Hama Machine List 33 | 34 | 35 | 36 |

<%=trackerName%> Hama Machine List

37 |
38 |

Grooms

39 |
40 | <% 41 | out.println(BSPServletUtil.generateGroomsTable(type, status, tracker)); 42 | out.println(BSPServletUtil.htmlFooter()); 43 | %> 44 |
-------------------------------------------------------------------------------- /core/src/main/resources/webapp/groomserver/index.html: -------------------------------------------------------------------------------- 1 | 17 | -------------------------------------------------------------------------------- /core/src/test/java/org/apache/hama/HamaCluster.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; 21 | 22 | import org.apache.commons.logging.Log; 23 | import org.apache.commons.logging.LogFactory; 24 | 25 | /** 26 | * Forming up the miniDfs and miniZooKeeper 27 | */ 28 | public abstract class HamaCluster extends HamaClusterTestCase { 29 | public static final Log LOG = LogFactory.getLog(HamaCluster.class); 30 | private final static HamaConfiguration conf = new HamaConfiguration(); 31 | 32 | public HamaCluster() { 33 | super(); 34 | } 35 | 36 | public HamaCluster(boolean startDfs) { 37 | super(startDfs); 38 | } 39 | 40 | @Override 41 | protected void setUp() throws Exception { 42 | super.setUp(); 43 | } 44 | 45 | protected static HamaConfiguration getConf() { 46 | return conf; 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /core/src/test/java/org/apache/hama/manager/TestLogView.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.manager; 19 | 20 | import junit.framework.TestCase; 21 | 22 | import org.apache.commons.logging.Log; 23 | import org.apache.commons.logging.LogFactory; 24 | import org.apache.hama.manager.util.UITemplate; 25 | 26 | public class TestLogView extends TestCase { 27 | 28 | public static final Log LOG = LogFactory.getLog(UITemplate.class); 29 | 30 | public void testconvertFileSize() throws Exception { 31 | 32 | assertEquals("File size 1024.", LogView.convertFileSize(1024), "1 KB"); 33 | assertEquals("File size 1048576.", LogView.convertFileSize(1048576), "1 MB"); 34 | assertEquals("File size 1073741824.", LogView.convertFileSize(1073741824), 35 | "1 GB"); 36 | assertEquals("File size 1099511627776L.", 37 | LogView.convertFileSize(1099511627776L), "1 TB"); 38 | assertEquals("File size 0.", LogView.convertFileSize(0), "0 bytes"); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /core/src/test/java/org/apache/hama/manager/util/TestUITemplate.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.manager.util; 19 | 20 | import junit.framework.TestCase; 21 | 22 | import org.apache.commons.logging.Log; 23 | import org.apache.commons.logging.LogFactory; 24 | 25 | public class TestUITemplate extends TestCase { 26 | 27 | public static final Log LOG = LogFactory.getLog(UITemplate.class); 28 | 29 | String tplPath = "webapp/commons/tpl/"; 30 | String tplFileName = "tpl.configlist.html"; 31 | 32 | public void testload() throws Exception { 33 | 34 | UITemplate uit = new UITemplate(); 35 | 36 | String tplfile = uit.load(tplPath + tplFileName); 37 | 38 | assertTrue(tplfile.length() > 0); 39 | } 40 | 41 | public void testGetArea() throws Exception { 42 | 43 | UITemplate uit = new UITemplate(); 44 | 45 | String tplfile = uit.load(tplPath + tplFileName); 46 | 47 | String tplHead = uit.getArea(tplfile, "head"); 48 | String tplTail = uit.getArea(tplfile, "tail"); 49 | 50 | assertTrue(tplHead.length() > 0); 51 | assertTrue(tplTail.length() > 0); 52 | } 53 | 54 | } 55 | -------------------------------------------------------------------------------- /core/src/test/java/org/apache/hama/monitor/TestConfigurator.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.monitor; 19 | 20 | import java.util.Map; 21 | 22 | import junit.framework.TestCase; 23 | 24 | import org.apache.commons.logging.Log; 25 | import org.apache.commons.logging.LogFactory; 26 | import org.apache.hama.HamaConfiguration; 27 | import org.apache.hama.monitor.Monitor.Task; 28 | 29 | public class TestConfigurator extends TestCase { 30 | 31 | public static final Log LOG = LogFactory.getLog(TestConfigurator.class); 32 | 33 | /** 34 | * If test fails, please check if `plugins' dir exists under "user.dir". 35 | */ 36 | public void testPluginDirNotPresented() throws Exception { 37 | System.setProperty("hama.home.dir", System.getProperty("user.dir")); 38 | Map tasks = Configurator.configure(new HamaConfiguration(), 39 | null); 40 | LOG.info("Plugins dir is not created, returned tasks should be null -> " 41 | + tasks); 42 | assertNull( 43 | "Tasks returned should be null because no plugins dir is created.", 44 | tasks); 45 | } 46 | 47 | } 48 | -------------------------------------------------------------------------------- /core/src/test/java/org/apache/hama/util/TestNumeric.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.util; 21 | 22 | import junit.framework.TestCase; 23 | 24 | public class TestNumeric extends TestCase { 25 | final static int TEST_INT = 3; 26 | final static double TEST_DOUBLE = 0.4; 27 | 28 | /** 29 | * Double conversion test 30 | */ 31 | public void testDouble() { 32 | assertEquals(Bytes.toDouble(Bytes.toBytes(TEST_DOUBLE)), TEST_DOUBLE); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /core/src/test/java/org/apache/hama/zookeeper/TestZKTools.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.zookeeper; 19 | 20 | import junit.framework.TestCase; 21 | 22 | import org.apache.hama.Constants; 23 | import org.apache.hama.HamaConfiguration; 24 | 25 | public class TestZKTools extends TestCase { 26 | 27 | public void testZKProps() { 28 | HamaConfiguration conf = new HamaConfiguration(); 29 | conf.set(Constants.ZOOKEEPER_QUORUM, "test.com:123"); 30 | conf.set(Constants.ZOOKEEPER_CLIENT_PORT, "2222"); 31 | 32 | assertEquals("test.com:2222", QuorumPeer.getZKQuorumServersString(conf)); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /core/src/test/resources/hama-metrics-msys.properties: -------------------------------------------------------------------------------- 1 | # syntax: [prefix].[source|sink].[instance].[options] 2 | 3 | test.sink.systemmonitor.class=org.apache.hama.metrics.SystemMonitorSink 4 | -------------------------------------------------------------------------------- /core/src/test/resources/hama-metrics-test-config.properties: -------------------------------------------------------------------------------- 1 | # syntax: [prefix].[source|sink].[instance].[options] 2 | 3 | bspmaster.sink.file_jvm.name=jvm 4 | bspmaster.sink.file_jvm.filename=bspmaster-jvm-metrics.out 5 | 6 | groom.sink.file.filename=groom-metrics.out 7 | 8 | -------------------------------------------------------------------------------- /docker/Dockerfile: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | FROM tianon/centos:6.5 17 | 18 | ENV HAMA_HOME /opt/hama 19 | ENV HAMA_VERSION 0.6.4 20 | 21 | RUN yum -y update; yum clean all 22 | RUN yum -y install epel-release; yum clean all 23 | RUN yum install -y wget \ 24 | which \ 25 | openssh-clients \ 26 | openssh-server \ 27 | curl \ 28 | tar \ 29 | system-config-services \ 30 | sudo 31 | 32 | RUN sed -ri 's/UsePAM yes/#UsePAM yes/g' /etc/ssh/sshd_config 33 | RUN sed -ri 's/#UsePAM no/UsePAM no/g' /etc/ssh/sshd_config 34 | 35 | RUN service sshd start 36 | 37 | # java 38 | RUN curl -LO 'http://download.oracle.com/otn-pub/java/jdk/7u71-b14/jdk-7u71-linux-x64.rpm' -H 'Cookie: oraclelicense=accept-securebackup-cookie' 39 | RUN rpm -i jdk-7u71-linux-x64.rpm 40 | RUN rm jdk-7u71-linux-x64.rpm 41 | ENV JAVA_HOME /usr/java/default 42 | ENV PATH $PATH:$JAVA_HOME/bin 43 | 44 | # hama 45 | RUN wget http://mirror.apache-kr.org/hama/hama-$HAMA_VERSION/hama-$HAMA_VERSION.tar.gz 46 | RUN tar -zxvf hama-$HAMA_VERSION.tar.gz 47 | RUN rm -rf hama-*.tar.gz 48 | RUN export HAMA_HOME=$HAMA_HOME 49 | RUN mv hama-* $HAMA_HOME 50 | 51 | COPY ./conf/hama-site.xml $HAMA_HOME/conf/ 52 | 53 | EXPOSE 40013 54 | -------------------------------------------------------------------------------- /docker/README.md: -------------------------------------------------------------------------------- 1 | # Hama Dockerfile # 2 | 3 | This directory contains Dockerfile of [Hama](http://hama.apache.org) for [Docker](https://www.docker.com/)'s. 4 | 5 | 6 | ## What is Hama? 7 | pache Hama is a general BSP computing engine on top of Hadoop, which was established in 2012 as a Top-Level Project of The Apache Software Foundation. It provides High-Performance computing engine for performing massive scientific and iterative algorithms on existing open source or enterprise Hadoop cluster, such as Matrix, Graph, and Machine Learning.[detail. https://hama.apache.org/](https://hama.apache.org/) 8 | 9 | 10 | ## Installation 11 | 12 | 1. Install [Docker](https://www.docker.com/). 13 | 14 | 2a. Build from files in this directory: 15 | Review hama version inside The Dockerfile 'ENV HAMA_VERSION' 16 | 17 | $docker build -t . 18 | ex) $docker build -t hama-docker . 19 | 20 | 2b. Download automated build from public hub registry: 21 | Pre-built image will down load and then you can run HAMA immediately. 22 | 23 | $docker pull hoseog/hama-docker 24 | 25 | 26 | ## Usage 27 | You can run hama with docker command and let you go into container. 28 | 29 | $docker run -it -name -p 40013:40013 /bin/bash 30 | ex) docker run -it -name hama-docker -p 40013:40013 hama-docker /bin/bash 31 | 32 | Hama is located in /opt/hama/ and is almost ready to run. 33 | Review configuration in /opt/hama/conf/ and you can start BST works. 34 | 35 | -------------------------------------------------------------------------------- /docs/diagram/hama-diagram.pptx: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/hama/734a4d7337cbd132623c7dcb84c300de428d13bf/docs/diagram/hama-diagram.pptx -------------------------------------------------------------------------------- /docs/diagram/yarn-hama-diagram.pptx: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/hama/734a4d7337cbd132623c7dcb84c300de428d13bf/docs/diagram/yarn-hama-diagram.pptx -------------------------------------------------------------------------------- /examples/src/test/java/org/apache/hama/examples/CombineExampleTest.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.examples; 19 | 20 | import static org.junit.Assert.fail; 21 | 22 | import org.junit.Test; 23 | 24 | /** 25 | * Testcase for {@link org.apache.hama.examples.CombineExample} 26 | */ 27 | public class CombineExampleTest { 28 | @Test 29 | public void testCorrectCombineExecution() { 30 | try { 31 | CombineExample.main(new String[] {}); 32 | } catch (Exception e) { 33 | fail(e.getLocalizedMessage()); 34 | } 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /examples/src/test/java/org/apache/hama/examples/LinearRegressionTest.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.examples; 19 | 20 | import org.junit.Test; 21 | 22 | /** 23 | * Testcase for {@link GradientDescentExample} for 'linear regression' 24 | */ 25 | public class LinearRegressionTest { 26 | @Test 27 | public void testCorrectGDWithLinearRegressionExecution() throws Exception { 28 | GradientDescentExample.main(new String[] { 29 | "src/test/resources/linear_regression_sample.txt", "linear" }); 30 | } 31 | 32 | } 33 | -------------------------------------------------------------------------------- /examples/src/test/java/org/apache/hama/examples/LogisticRegressionTest.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.examples; 19 | 20 | import org.junit.Test; 21 | 22 | /** 23 | * Testcase for {@link GradientDescentExample} execution for 'logistic 24 | * regression' 25 | */ 26 | public class LogisticRegressionTest { 27 | 28 | @Test 29 | public void testCorrectGDWithLogisticRegressionExecution() throws Exception { 30 | GradientDescentExample.main(new String[] { 31 | "src/test/resources/logistic_regression_sample.txt", "logistic" }); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /examples/src/test/java/org/apache/hama/examples/PiEstimatorTest.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.examples; 19 | 20 | import static org.junit.Assert.fail; 21 | 22 | import org.junit.Test; 23 | 24 | /** 25 | * Testcase for {@link PiEstimator} 26 | */ 27 | public class PiEstimatorTest { 28 | @Test 29 | public void testCorrectPiExecution() { 30 | try { 31 | PiEstimator.main(new String[] { "10" }); 32 | } catch (Exception e) { 33 | fail(e.getLocalizedMessage()); 34 | } 35 | } 36 | 37 | @Test 38 | public void testPiExecutionWithEmptyArgs() { 39 | try { 40 | PiEstimator.main(new String[10]); 41 | fail("PiEstimator should fail if the argument list has size 0"); 42 | } catch (Exception e) { 43 | // everything ok 44 | } 45 | } 46 | 47 | } 48 | -------------------------------------------------------------------------------- /examples/src/test/java/org/apache/hama/examples/RandBenchTest.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.examples; 19 | 20 | import static org.junit.Assert.fail; 21 | 22 | import org.junit.Test; 23 | 24 | /** 25 | * Testcase for {@link org.apache.hama.examples.RandBench} 26 | */ 27 | public class RandBenchTest { 28 | @Test 29 | public void testCorrectRandBenchExecution() { 30 | try { 31 | RandBench.main(new String[] { "10", "3", "2" }); 32 | } catch (Exception e) { 33 | fail(e.getLocalizedMessage()); 34 | } 35 | } 36 | 37 | @Test 38 | public void testRandBenchExecutionWithEmptyArgs() { 39 | try { 40 | RandBench.main(new String[10]); 41 | fail("RandBench should fail if the argument list has size < 3"); 42 | } catch (Exception e) { 43 | // everything ok 44 | } 45 | } 46 | 47 | } 48 | -------------------------------------------------------------------------------- /examples/src/test/resources/dg.txt: -------------------------------------------------------------------------------- 1 | 1 2 | 2 3 | 3 4 | 4 -------------------------------------------------------------------------------- /examples/src/test/resources/kcore.txt: -------------------------------------------------------------------------------- 1 | 1 2 2 | 2 1 3 9 8 5 3 | 3 2 4 | 4 5 5 | 5 2 4 6 8 9 6 | 6 5 7 8 7 | 7 6 8 | 8 2 5 6 9 10 11 9 | 9 2 5 8 10 | 10 8 11 11 | 11 8 10 -------------------------------------------------------------------------------- /examples/src/test/resources/linear_regression_sample.txt: -------------------------------------------------------------------------------- 1 | 0.1>0.1 0.2 0.3 0.4 0.5 0.6 0.7 2 | 0.12>0.3 0.4 0.5 0.6 0.9 0.1 0.3 3 | 0.22>0.1 0.1 0.3 0.1 0.1 0.01 0.1 4 | 0.4>0.2 0.4 0.1 0.1 0.4 0.1 0.8 5 | 0.3>0.3 0.4 0.5 0.6 0.7 0.8 0.9 6 | 0.71>0.1 0.3 0.4 0.1 0.4 0.5 0.1 7 | 0.12>0.5 0.7 0.1 0.6 0.7 0.1 0 8 | 0.4>0.1 0.01 0.6 0.99 0.1 0.1 0.5 -------------------------------------------------------------------------------- /examples/src/test/resources/logistic_regression_sample.txt: -------------------------------------------------------------------------------- 1 | 0>1 9 2 4 5 6 7 2 | 1>3 4 5 6 9 1 3 3 | 1>1 1 3 1 1 1 1 4 | 0>2 4 1 1 4 1 8 5 | 1>3 4 5 6 7 8 9 6 | 1>1 3 4 1 4 5 1 7 | 0>1 10 3 2 1 6 1 8 | 0>1 2 2 2 2 6 2 -------------------------------------------------------------------------------- /graph/src/main/java/org/apache/hama/graph/Aggregator.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.graph; 19 | 20 | import org.apache.hadoop.io.Writable; 21 | 22 | /** 23 | * Aggregators are a mechanism for global communication, monitoring, and data. 24 | * Each vertex can provide a value to an aggregator in superstep S, the system 25 | * combines those values using a reduction operator, and the resulting value is 26 | * made available to all vertices in superstep S + 1.
27 | * The result of an aggregator from the last superstep can be picked up by the 28 | * vertex itself via {@link Vertex}#getLastAggregatedValue(); 29 | */ 30 | public interface Aggregator { 31 | 32 | /** 33 | * Observes a new vertex value. 34 | */ 35 | public void aggregate(M value); 36 | 37 | /** 38 | * Gets a vertex value. 39 | */ 40 | public M getValue(); 41 | 42 | } 43 | -------------------------------------------------------------------------------- /graph/src/main/java/org/apache/hama/graph/AverageAggregator.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.graph; 19 | 20 | import org.apache.hadoop.io.DoubleWritable; 21 | 22 | /** 23 | * Averages the result of the {@link AbsDiffAggregator}. 24 | */ 25 | public class AverageAggregator extends AbsDiffAggregator { 26 | 27 | @Override 28 | public DoubleWritable finalizeAggregation() { 29 | return new DoubleWritable(getValue().get() / getTimesAggregated().get()); 30 | } 31 | 32 | } 33 | -------------------------------------------------------------------------------- /graph/src/main/java/org/apache/hama/graph/MaxAggregator.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.graph; 19 | 20 | import org.apache.hadoop.io.IntWritable; 21 | 22 | public class MaxAggregator extends 23 | AbstractAggregator { 24 | 25 | int max = Integer.MIN_VALUE; 26 | 27 | @Override 28 | public void aggregate(IntWritable value) { 29 | if (value.get() > max) { 30 | max = value.get(); 31 | } 32 | } 33 | 34 | @Override 35 | public IntWritable getValue() { 36 | return new IntWritable(max); 37 | } 38 | 39 | } 40 | -------------------------------------------------------------------------------- /graph/src/main/java/org/apache/hama/graph/MinAggregator.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.graph; 19 | 20 | import org.apache.hadoop.io.IntWritable; 21 | 22 | public class MinAggregator extends 23 | AbstractAggregator { 24 | 25 | int min = Integer.MAX_VALUE; 26 | 27 | @Override 28 | public void aggregate(IntWritable value) { 29 | if (value.get() < min) { 30 | min = value.get(); 31 | } 32 | } 33 | 34 | @Override 35 | public IntWritable getValue() { 36 | return new IntWritable(min); 37 | } 38 | 39 | } 40 | -------------------------------------------------------------------------------- /graph/src/main/java/org/apache/hama/graph/SumAggregator.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.graph; 19 | 20 | import org.apache.hadoop.io.DoubleWritable; 21 | 22 | /** 23 | * Sums all vertex values globally. 24 | */ 25 | public class SumAggregator extends 26 | AbstractAggregator { 27 | 28 | double sum = 0.0d; 29 | 30 | @Override 31 | public void aggregate(DoubleWritable value) { 32 | sum += value.get(); 33 | } 34 | 35 | @Override 36 | public DoubleWritable getValue() { 37 | return new DoubleWritable(sum); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /graph/src/main/java/org/apache/hama/graph/package.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 19 | 20 | 21 | Pregel clone graph computing framework on top of Hama. 22 | 23 | 24 | -------------------------------------------------------------------------------- /graph/src/test/java/org/apache/hama/graph/TestAbsDiffAggregator.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.graph; 19 | 20 | import junit.framework.TestCase; 21 | 22 | import org.apache.hadoop.io.DoubleWritable; 23 | import org.junit.Test; 24 | 25 | public class TestAbsDiffAggregator extends TestCase { 26 | 27 | @Test 28 | public void testAggregator() { 29 | AbsDiffAggregator diff = new AbsDiffAggregator(); 30 | diff.aggregate(new DoubleWritable(5), new DoubleWritable(2)); 31 | diff.aggregate(new DoubleWritable(5), new DoubleWritable(2)); 32 | diff.aggregate(null, new DoubleWritable(5)); 33 | 34 | // 0, because this is totally worthless for diffs 35 | assertEquals(0, diff.getTimesAggregated().get()); 36 | assertEquals(6, (int) diff.getValue().get()); 37 | 38 | } 39 | 40 | } 41 | -------------------------------------------------------------------------------- /graph/src/test/java/org/apache/hama/graph/TestAverageAggregator.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.graph; 19 | 20 | import junit.framework.TestCase; 21 | 22 | import org.apache.hadoop.io.DoubleWritable; 23 | import org.junit.Test; 24 | 25 | public class TestAverageAggregator extends TestCase { 26 | 27 | @Test 28 | public void testAggregator() { 29 | AverageAggregator diff = new AverageAggregator(); 30 | diff.aggregate(new DoubleWritable(5), new DoubleWritable(2)); 31 | diff.aggregateInternal(); 32 | diff.aggregate(new DoubleWritable(5), new DoubleWritable(2)); 33 | diff.aggregateInternal(); 34 | diff.aggregate(null, new DoubleWritable(5)); 35 | diff.aggregateInternal(); 36 | 37 | assertEquals(3, diff.getTimesAggregated().get()); 38 | DoubleWritable x = diff.finalizeAggregation(); 39 | assertEquals(2, (int) x.get()); 40 | 41 | } 42 | 43 | } 44 | -------------------------------------------------------------------------------- /graph/src/test/java/org/apache/hama/graph/TestMinMaxAggregator.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.graph; 19 | 20 | import junit.framework.TestCase; 21 | 22 | import org.apache.hadoop.io.IntWritable; 23 | import org.junit.Test; 24 | 25 | public class TestMinMaxAggregator extends TestCase { 26 | 27 | @Test 28 | public void testMinAggregator() { 29 | MinAggregator diff = new MinAggregator(); 30 | diff.aggregate(new IntWritable(5)); 31 | diff.aggregate(new IntWritable(25)); 32 | assertEquals(5, diff.getValue().get()); 33 | 34 | } 35 | 36 | @Test 37 | public void testMaxAggregator() { 38 | MaxAggregator diff = new MaxAggregator(); 39 | diff.aggregate(new IntWritable(5)); 40 | diff.aggregate(new IntWritable(25)); 41 | assertEquals(25, diff.getValue().get()); 42 | } 43 | 44 | } 45 | -------------------------------------------------------------------------------- /graph/src/test/java/org/apache/hama/graph/TestSumAggregator.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.graph; 19 | 20 | import junit.framework.TestCase; 21 | 22 | import org.apache.hadoop.io.DoubleWritable; 23 | import org.junit.Test; 24 | 25 | public class TestSumAggregator extends TestCase { 26 | 27 | @Test 28 | public void testAggregator() { 29 | SumAggregator diff = new SumAggregator(); 30 | diff.aggregate(new DoubleWritable(5)); 31 | diff.aggregate(new DoubleWritable(5)); 32 | assertEquals(10, (int) diff.getValue().get()); 33 | 34 | } 35 | 36 | } 37 | -------------------------------------------------------------------------------- /mesos/.gitignore: -------------------------------------------------------------------------------- 1 | /target 2 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/classification/ClassifierConfigurationAdapter.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.ml.classification; 19 | 20 | /** 21 | * Adapter interface for classifier's configurations and parameters. 22 | */ 23 | public interface ClassifierConfigurationAdapter> { 24 | 25 | /** 26 | * applies the underlying adapted configuration to the given classifier 27 | * 28 | * @return a new {@link ClassifierAdapter} instance with the applied configuration 29 | */ 30 | public C applyConfiguration(C classifier); 31 | } 32 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/distance/DistanceMeasurer.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.ml.distance; 19 | 20 | import org.apache.hama.commons.math.DoubleVector; 21 | 22 | /** 23 | * a {@link DistanceMeasurer} is responsible for calculating the distance 24 | * between {@link DoubleVector}s or Arrays of {@code double}s 25 | */ 26 | public interface DistanceMeasurer { 27 | 28 | /** 29 | * Calculates the distance between two arrays of {@code double}s 30 | * 31 | * @param set1 an array of {@code double} 32 | * @param set2 an array of {@code double} 33 | * @return a {@code double} representing the distance 34 | */ 35 | public double measureDistance(double[] set1, double[] set2); 36 | 37 | /** 38 | * Calculates the distance between two {@link DoubleVector}ss 39 | * 40 | * @param vec1 a {@link DoubleVector} 41 | * @param vec2 a {@link DoubleVector} 42 | * @return a {@code double} representing the distance 43 | */ 44 | public double measureDistance(DoubleVector vec1, DoubleVector vec2); 45 | 46 | } 47 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/distance/EuclidianDistance.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.ml.distance; 19 | 20 | import org.apache.hama.commons.math.DoubleVector; 21 | 22 | public final class EuclidianDistance implements DistanceMeasurer { 23 | 24 | @Override 25 | public double measureDistance(double[] set1, double[] set2) { 26 | double sum = 0; 27 | int length = set1.length; 28 | for (int i = 0; i < length; i++) { 29 | double diff = set2[i] - set1[i]; 30 | // multiplication is faster than Math.pow() for ^2. 31 | sum += (diff * diff); 32 | } 33 | 34 | return Math.sqrt(sum); 35 | } 36 | 37 | @Override 38 | public double measureDistance(DoubleVector vec1, DoubleVector vec2) { 39 | return Math.sqrt(vec2.subtractUnsafe(vec1).pow(2).sum()); 40 | } 41 | 42 | } 43 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/kcore/KCoreMessage.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.ml.kcore; 19 | 20 | import java.io.DataInput; 21 | import java.io.DataOutput; 22 | import java.io.IOException; 23 | 24 | import org.apache.hadoop.io.Writable; 25 | 26 | public class KCoreMessage implements Writable { 27 | private long vertexID; 28 | private int core; 29 | 30 | public KCoreMessage() { 31 | this.vertexID = 0L; 32 | this.core = 0; 33 | } 34 | 35 | public KCoreMessage(long vertexID, int core) { 36 | this.vertexID = vertexID; 37 | this.core = core; 38 | } 39 | 40 | @Override 41 | public void readFields(DataInput input) throws IOException { 42 | vertexID = input.readLong(); 43 | core = input.readInt(); 44 | } 45 | 46 | @Override 47 | public void write(DataOutput output) throws IOException { 48 | output.writeLong(vertexID); 49 | output.writeInt(core); 50 | } 51 | 52 | public long getVertexID() { 53 | return vertexID; 54 | } 55 | 56 | public int getCore() { 57 | return core; 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/kcore/KCoreVertexWriter.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.ml.kcore; 19 | 20 | import java.io.IOException; 21 | 22 | import org.apache.hadoop.conf.Configuration; 23 | import org.apache.hadoop.io.IntWritable; 24 | import org.apache.hadoop.io.LongWritable; 25 | import org.apache.hadoop.io.Writable; 26 | import org.apache.hama.bsp.BSPPeer; 27 | import org.apache.hama.graph.GraphJobMessage; 28 | import org.apache.hama.graph.Vertex; 29 | import org.apache.hama.graph.VertexOutputWriter; 30 | 31 | public class KCoreVertexWriter 32 | implements 33 | VertexOutputWriter { 34 | 35 | @Override 36 | public void setup(Configuration conf) { 37 | 38 | } 39 | 40 | @Override 41 | public void write( 42 | Vertex vertex, 43 | BSPPeer peer) 44 | throws IOException { 45 | peer.write(vertex.getVertexID(), 46 | new IntWritable(((KCoreVertex) vertex).getCore())); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/recommendation/ItemSimilarity.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.ml.recommendation; 19 | 20 | import java.util.List; 21 | 22 | import org.apache.commons.math3.util.Pair; 23 | 24 | public interface ItemSimilarity { 25 | /** 26 | * calculate similarity between two items 27 | * @param item1 - first item 28 | * @param item2 - second item 29 | * @return item similarity, 0 == similar item 30 | */ 31 | double calculateItemSimilarity(long item1, long item2); 32 | 33 | /** 34 | * get most similar users 35 | * @param item - item id 36 | * @param count - number of similar items 37 | * @return list of similar item ids(key) and similarity(value) 38 | */ 39 | List> getMostSimilarItems(long item, int count); 40 | } 41 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/recommendation/RecommenderIO.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.ml.recommendation; 19 | 20 | public interface RecommenderIO { 21 | /** 22 | * set preferences input data 23 | * @param path - file path to preferences data 24 | * 25 | */ 26 | void setInputPreferences(String path); 27 | 28 | /** 29 | * set user features input data 30 | * @param path - file path to user features data if any 31 | * 32 | */ 33 | void setInputUserFeatures(String path); 34 | 35 | /** 36 | * set item features input data 37 | * @param path - file path to item features data if any 38 | * 39 | */ 40 | void setInputItemFeatures(String path); 41 | 42 | /** 43 | * set output path for trained model 44 | * @param path - output path 45 | * 46 | */ 47 | void setOutputPath(String path); 48 | 49 | } 50 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/recommendation/UserSimilarity.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.ml.recommendation; 19 | 20 | import java.util.List; 21 | 22 | import org.apache.commons.math3.util.Pair; 23 | 24 | public interface UserSimilarity { 25 | /** 26 | * calculate similarity between two users 27 | * @param user1 - first user 28 | * @param user2 - second user 29 | * @return user similarity, 0 == similar user 30 | */ 31 | double calculateUserSimilarity(long user1, long user2); 32 | 33 | /** 34 | * get most similar users 35 | * @param user - user id 36 | * @param count - number of similar users 37 | * @return list of similar user ids(key) and similarity(value) 38 | */ 39 | List> getMostSimilarUsers(long user, int count); 40 | } 41 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/recommendation/cf/KeyValueParser.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.ml.recommendation.cf; 19 | 20 | import org.apache.hadoop.io.Writable; 21 | 22 | /** 23 | * Take line and convert it to Key Value, 24 | * class contains "K key" 25 | * and "V value" variables 26 | * 27 | * @param - key 28 | * @param - value 29 | */ 30 | public abstract class KeyValueParser { 31 | protected K key = null; 32 | protected V value = null; 33 | public abstract void parseLine(String ln); 34 | 35 | /** 36 | * 37 | * @return key, if not parsed yet may return null 38 | * or value from last parsed line 39 | */ 40 | public K getKey() { 41 | return key; 42 | } 43 | 44 | /** 45 | * 46 | * @return value, if not parsed yet may return null 47 | * or value from last parsed line 48 | */ 49 | public V getValue() { 50 | return value; 51 | } 52 | } 53 | 54 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/regression/CostFunction.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.ml.regression; 19 | 20 | import java.math.BigDecimal; 21 | 22 | import org.apache.hama.commons.math.DoubleVector; 23 | 24 | /** 25 | * An optimization (minimization) problem's cost function 26 | */ 27 | public interface CostFunction { 28 | 29 | /** 30 | * Calculates the cost function for a given item (input x, output y), a model 31 | * defined by the hypothesis parametrized by the vector theta 32 | * 33 | * @param x the input vector 34 | * @param y the learned output for x 35 | * @param m the number of existing items 36 | * @param theta the parameters vector theta 37 | * @param hypothesis the hypothesis function to model the problem 38 | * @return the calculated cost for input x and output y as a BigDecimal 39 | */ 40 | public BigDecimal calculateCostForItem(DoubleVector x, double y, int m, 41 | DoubleVector theta, HypothesisFunction hypothesis); 42 | 43 | } 44 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/regression/HypothesisFunction.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.ml.regression; 19 | 20 | import java.math.BigDecimal; 21 | 22 | import org.apache.hama.commons.math.DoubleVector; 23 | 24 | /** 25 | * The mathematical model chosen for a specific learning problem 26 | */ 27 | public interface HypothesisFunction { 28 | 29 | /** 30 | * Applies this HypothesisFunction to given a set of parameters theta and 31 | * input x 32 | * 33 | * @param theta the parameters vector 34 | * @param x the input 35 | * @return a BigDecimal representing the number 36 | */ 37 | public BigDecimal applyHypothesis(DoubleVector theta, DoubleVector x); 38 | 39 | } 40 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/regression/RegressionModel.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.ml.regression; 19 | 20 | import java.math.BigDecimal; 21 | 22 | import org.apache.hama.commons.math.DoubleVector; 23 | 24 | /** 25 | * A cost model for gradient descent based regression 26 | */ 27 | public interface RegressionModel extends HypothesisFunction { 28 | 29 | /** 30 | * Calculates the cost function for a given item (input x, output y) and the 31 | * model's parameters defined by the vector theta 32 | * 33 | * @param x the input vector 34 | * @param y the learned output for x 35 | * @param m the total number of existing items 36 | * @param theta the parameters vector theta 37 | * @return the calculated cost for input x and output y as a BigDecimal 38 | */ 39 | public BigDecimal calculateCostForItem(DoubleVector x, double y, int m, 40 | DoubleVector theta); 41 | 42 | } 43 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/util/DefaultFeatureTransformer.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.ml.util; 19 | 20 | import org.apache.hama.commons.math.DoubleVector; 21 | 22 | /** 23 | * The defaultFeatureTransformer actually returns the original features. 24 | */ 25 | public class DefaultFeatureTransformer extends FeatureTransformer { 26 | 27 | /** 28 | * Directly return the original features. 29 | */ 30 | @Override 31 | public DoubleVector transform(DoubleVector originalFeatures) { 32 | return originalFeatures; 33 | } 34 | 35 | } 36 | -------------------------------------------------------------------------------- /ml/src/main/java/org/apache/hama/ml/util/FeatureTransformer.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.ml.util; 19 | 20 | import org.apache.hama.commons.math.DoubleVector; 21 | 22 | /** 23 | * FeatureTransformer defines the interface to transform the original features 24 | * to new space. 25 | * 26 | * NOTE: the user defined feature transformer must have a constructor with no parameters. 27 | * 28 | */ 29 | public abstract class FeatureTransformer { 30 | 31 | public FeatureTransformer() { 32 | } 33 | 34 | /** 35 | * Transform the original features to transformed space. 36 | * @param originalFeatures 37 | * @return a new vector with the result of the operation. 38 | */ 39 | public abstract DoubleVector transform(DoubleVector originalFeatures); 40 | 41 | } 42 | -------------------------------------------------------------------------------- /ml/src/test/resources/vd_file_sample.txt: -------------------------------------------------------------------------------- 1 | 1>2 3 4 2 | 5>6 7 8 -------------------------------------------------------------------------------- /python/BSP.py: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | """ 17 | 18 | BSP Class that can be overridden to implement the computation logic. 19 | 20 | """ 21 | from BSPPeer import BSPPeer 22 | 23 | class BSP: 24 | 25 | def __init__(self): 26 | pass 27 | 28 | def setup(self, peer): 29 | pass 30 | 31 | def bsp(self, peer): 32 | pass 33 | 34 | def cleanup(self, peer): 35 | pass -------------------------------------------------------------------------------- /python/BSPRunner.py: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | """ 17 | 18 | Main Runner utility that will get the bsp class from the user, passed via args and start 19 | it with the whole context and stuff. 20 | 21 | """ 22 | import sys 23 | from BSPPeer import BSPPeer 24 | 25 | className = sys.argv[1] 26 | module = __import__(className) 27 | class_ = getattr(module, className) 28 | 29 | bspInstance = class_() 30 | 31 | peer = BSPPeer(bspInstance) 32 | peer.runSetup() 33 | peer.runBSP() 34 | peer.runCleanup() 35 | peer.done() -------------------------------------------------------------------------------- /python/BspJobConfiguration.py: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | """ 17 | 18 | A mimic configuration object contains a dictionary that maps keys to values to store information. 19 | 20 | """ 21 | class BspJobConfiguration: 22 | def __init__(self): 23 | self.conf = {} 24 | 25 | def get(self, key): 26 | return self.conf[key] 27 | 28 | def put(self, key, value): 29 | self.conf[key] = value -------------------------------------------------------------------------------- /python/HelloWorldBSP.py: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | """ 17 | 18 | Basic Hello World BSP, in Hama this is called serialize printing. 19 | Each task sends its peer name to each other task who reads the 20 | message and outputs it to console. 21 | 22 | """ 23 | from BSP import BSP 24 | 25 | class HelloWorldBSP(BSP): 26 | def bsp(self, peer): 27 | name = peer.getPeerName() 28 | for i in range(15): 29 | for otherPeer in peer.getAllPeerNames(): 30 | peer.send(otherPeer, ("Hello from " + name + " in superstep " + str(i))) 31 | peer.sync() 32 | for msg in peer.getAllMessages(): 33 | peer.write(msg,"") 34 | 35 | 36 | -------------------------------------------------------------------------------- /python/README.txt: -------------------------------------------------------------------------------- 1 | Hama Streaming protocol implementation for Python. 2 | Features the usual hello world for BSP and a simple k means clustering. 3 | 4 | Using Python 3.2.3 and PyCharm 2.6. 5 | 6 | Documentation can be found here: 7 | 8 | http://wiki.apache.org/hama/HamaStreaming 9 | 10 | Protocol documentation can be found here: 11 | 12 | http://wiki.apache.org/hama/StreamingProtocol -------------------------------------------------------------------------------- /src/site/resources/images/apache-hama-white.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/hama/734a4d7337cbd132623c7dcb84c300de428d13bf/src/site/resources/images/apache-hama-white.png -------------------------------------------------------------------------------- /src/site/resources/images/favicon.ico: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/hama/734a4d7337cbd132623c7dcb84c300de428d13bf/src/site/resources/images/favicon.ico -------------------------------------------------------------------------------- /src/site/resources/images/hama_art_arthur.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/hama/734a4d7337cbd132623c7dcb84c300de428d13bf/src/site/resources/images/hama_art_arthur.png -------------------------------------------------------------------------------- /src/site/resources/images/hama_art_arthur_300x220.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/hama/734a4d7337cbd132623c7dcb84c300de428d13bf/src/site/resources/images/hama_art_arthur_300x220.png -------------------------------------------------------------------------------- /src/site/resources/images/hama_kim.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/hama/734a4d7337cbd132623c7dcb84c300de428d13bf/src/site/resources/images/hama_kim.png -------------------------------------------------------------------------------- /src/site/resources/images/hama_logo.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/hama/734a4d7337cbd132623c7dcb84c300de428d13bf/src/site/resources/images/hama_logo.png -------------------------------------------------------------------------------- /src/site/resources/images/hama_paint_logo.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/hama/734a4d7337cbd132623c7dcb84c300de428d13bf/src/site/resources/images/hama_paint_logo.png -------------------------------------------------------------------------------- /src/site/resources/images/hama_paint_logo_120x120.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/hama/734a4d7337cbd132623c7dcb84c300de428d13bf/src/site/resources/images/hama_paint_logo_120x120.png -------------------------------------------------------------------------------- /src/site/resources/images/headerlogo.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/hama/734a4d7337cbd132623c7dcb84c300de428d13bf/src/site/resources/images/headerlogo.png -------------------------------------------------------------------------------- /src/site/resources/images/headerlogo.xcf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/hama/734a4d7337cbd132623c7dcb84c300de428d13bf/src/site/resources/images/headerlogo.xcf -------------------------------------------------------------------------------- /src/site/resources/images/mahout_vs_hama.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/hama/734a4d7337cbd132623c7dcb84c300de428d13bf/src/site/resources/images/mahout_vs_hama.png -------------------------------------------------------------------------------- /src/site/xdoc/artwork.xml: -------------------------------------------------------------------------------- 1 | 2 | 18 | 21 | 22 | Artwork and Clothing 23 | 24 | 25 | 26 |
27 | 28 | 29 | 30 | 31 | 32 | 33 |
34 | -------------------------------------------------------------------------------- /src/site/xdoc/irc-channel.xml: -------------------------------------------------------------------------------- 1 | 2 | 18 | 21 | 22 | IRC Channel 23 | 24 | 25 |
26 |

There is an IRC channel dedicated to Hama at irc.freenode.org. The name of the channel is #hama. 27 |

28 | The IRC channel can be used for online discussion about hama related stuff, but developers should be 29 | careful to transfer all the official decisions or useful discussions to the issue tracking system. 30 |

31 | 32 |
33 | -------------------------------------------------------------------------------- /src/site/xdoc/issue-tracking.xml: -------------------------------------------------------------------------------- 1 | 2 | 18 | 21 | 22 | Issue Tracking 23 | 24 | 25 |
26 |

Hama tracks both bugs and enhancement requests 27 | here using Apache JIRA. 28 | We welcome input, however, before filing a request, please make sure you do the following:

29 |
    30 |
  • Search the JIRA database.
  • 31 |
  • Check the user mailing list, both by searching the archives and by asking questions.
  • 32 |
33 | 34 |
35 | -------------------------------------------------------------------------------- /yarn/src/main/java/org/apache/hama/bsp/BSPClient.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.LongWritable; 21 | import org.apache.hama.ipc.VersionedProtocol; 22 | 23 | public interface BSPClient extends VersionedProtocol { 24 | 25 | public static final int versionID = 1; 26 | 27 | public LongWritable getCurrentSuperStep(); 28 | 29 | } 30 | --------------------------------------------------------------------------------