|
| 1 | +package spark.examples |
| 2 | + |
| 3 | +import org.apache.hadoop.mapreduce.Job |
| 4 | +import org.apache.cassandra.hadoop.ColumnFamilyOutputFormat |
| 5 | +import org.apache.cassandra.hadoop.ConfigHelper |
| 6 | +import org.apache.cassandra.hadoop.ColumnFamilyInputFormat |
| 7 | +import org.apache.cassandra.thrift._ |
| 8 | +import spark.SparkContext |
| 9 | +import spark.SparkContext._ |
| 10 | +import java.nio.ByteBuffer |
| 11 | +import java.util.SortedMap |
| 12 | +import org.apache.cassandra.db.IColumn |
| 13 | +import org.apache.cassandra.utils.ByteBufferUtil |
| 14 | +import scala.collection.JavaConversions._ |
| 15 | + |
| 16 | + |
| 17 | +/* |
| 18 | + * This example demonstrates using Spark with Cassandra with the New Hadoop API and Cassandra |
| 19 | + * support for Hadoop. |
| 20 | + * |
| 21 | + * To run this example, run this file with the following command params - |
| 22 | + * <spark_master> <cassandra_node> <cassandra_port> |
| 23 | + * |
| 24 | + * So if you want to run this on localhost this will be, |
| 25 | + * local[3] localhost 9160 |
| 26 | + * |
| 27 | + * The example makes some assumptions: |
| 28 | + * 1. You have already created a keyspace called casDemo and it has a column family named Words |
| 29 | + * 2. There are column family has a column named "para" which has test content. |
| 30 | + * |
| 31 | + * You can create the content by running the following script at the bottom of this file with |
| 32 | + * cassandra-cli. |
| 33 | + * |
| 34 | + */ |
| 35 | +object CassandraTest { |
| 36 | + |
| 37 | + def main(args: Array[String]) { |
| 38 | + |
| 39 | + // Get a SparkContext |
| 40 | + val sc = new SparkContext(args(0), "casDemo") |
| 41 | + |
| 42 | + // Build the job configuration with ConfigHelper provided by Cassandra |
| 43 | + val job = new Job() |
| 44 | + job.setInputFormatClass(classOf[ColumnFamilyInputFormat]) |
| 45 | + |
| 46 | + val host: String = args(1) |
| 47 | + val port: String = args(2) |
| 48 | + |
| 49 | + ConfigHelper.setInputInitialAddress(job.getConfiguration(), host) |
| 50 | + ConfigHelper.setInputRpcPort(job.getConfiguration(), port) |
| 51 | + ConfigHelper.setOutputInitialAddress(job.getConfiguration(), host) |
| 52 | + ConfigHelper.setOutputRpcPort(job.getConfiguration(), port) |
| 53 | + ConfigHelper.setInputColumnFamily(job.getConfiguration(), "casDemo", "Words") |
| 54 | + ConfigHelper.setOutputColumnFamily(job.getConfiguration(), "casDemo", "WordCount") |
| 55 | + |
| 56 | + val predicate = new SlicePredicate() |
| 57 | + val sliceRange = new SliceRange() |
| 58 | + sliceRange.setStart(Array.empty[Byte]) |
| 59 | + sliceRange.setFinish(Array.empty[Byte]) |
| 60 | + predicate.setSlice_range(sliceRange) |
| 61 | + ConfigHelper.setInputSlicePredicate(job.getConfiguration(), predicate) |
| 62 | + |
| 63 | + ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner") |
| 64 | + ConfigHelper.setOutputPartitioner(job.getConfiguration(), "Murmur3Partitioner") |
| 65 | + |
| 66 | + // Make a new Hadoop RDD |
| 67 | + val casRdd = sc.newAPIHadoopRDD( |
| 68 | + job.getConfiguration(), |
| 69 | + classOf[ColumnFamilyInputFormat], |
| 70 | + classOf[ByteBuffer], |
| 71 | + classOf[SortedMap[ByteBuffer, IColumn]]) |
| 72 | + |
| 73 | + // Let us first get all the paragraphs from the retrieved rows |
| 74 | + val paraRdd = casRdd.map { |
| 75 | + case (key, value) => { |
| 76 | + ByteBufferUtil.string(value.get(ByteBufferUtil.bytes("para")).value()) |
| 77 | + } |
| 78 | + } |
| 79 | + |
| 80 | + // Lets get the word count in paras |
| 81 | + val counts = paraRdd.flatMap(p => p.split(" ")).map(word => (word, 1)).reduceByKey(_ + _) |
| 82 | + |
| 83 | + counts.collect().foreach { |
| 84 | + case (word, count) => println(word + ":" + count) |
| 85 | + } |
| 86 | + |
| 87 | + counts.map { |
| 88 | + case (word, count) => { |
| 89 | + val colWord = new org.apache.cassandra.thrift.Column() |
| 90 | + colWord.setName(ByteBufferUtil.bytes("word")) |
| 91 | + colWord.setValue(ByteBufferUtil.bytes(word)) |
| 92 | + colWord.setTimestamp(System.currentTimeMillis) |
| 93 | + |
| 94 | + val colCount = new org.apache.cassandra.thrift.Column() |
| 95 | + colCount.setName(ByteBufferUtil.bytes("wcount")) |
| 96 | + colCount.setValue(ByteBufferUtil.bytes(count.toLong)) |
| 97 | + colCount.setTimestamp(System.currentTimeMillis) |
| 98 | + |
| 99 | + val outputkey = ByteBufferUtil.bytes(word + "-COUNT-" + System.currentTimeMillis) |
| 100 | + |
| 101 | + val mutations: java.util.List[Mutation] = new Mutation() :: new Mutation() :: Nil |
| 102 | + mutations.get(0).setColumn_or_supercolumn(new ColumnOrSuperColumn()) |
| 103 | + mutations.get(0).column_or_supercolumn.setColumn(colWord) |
| 104 | + mutations.get(1).setColumn_or_supercolumn(new ColumnOrSuperColumn()) |
| 105 | + mutations.get(1).column_or_supercolumn.setColumn(colCount) |
| 106 | + (outputkey, mutations) |
| 107 | + } |
| 108 | + }.saveAsNewAPIHadoopFile("casDemo", classOf[ByteBuffer], classOf[List[Mutation]], |
| 109 | + classOf[ColumnFamilyOutputFormat], job.getConfiguration) |
| 110 | + } |
| 111 | +} |
| 112 | + |
| 113 | +/* |
| 114 | +create keyspace casDemo; |
| 115 | +use casDemo; |
| 116 | +
|
| 117 | +create column family WordCount with comparator = UTF8Type; |
| 118 | +update column family WordCount with column_metadata = |
| 119 | + [{column_name: word, validation_class: UTF8Type}, |
| 120 | + {column_name: wcount, validation_class: LongType}]; |
| 121 | +
|
| 122 | +create column family Words with comparator = UTF8Type; |
| 123 | +update column family Words with column_metadata = |
| 124 | + [{column_name: book, validation_class: UTF8Type}, |
| 125 | + {column_name: para, validation_class: UTF8Type}]; |
| 126 | +
|
| 127 | +assume Words keys as utf8; |
| 128 | +
|
| 129 | +set Words['3musk001']['book'] = 'The Three Musketeers'; |
| 130 | +set Words['3musk001']['para'] = 'On the first Monday of the month of April, 1625, the market |
| 131 | + town of Meung, in which the author of ROMANCE OF THE ROSE was born, appeared to |
| 132 | + be in as perfect a state of revolution as if the Huguenots had just made |
| 133 | + a second La Rochelle of it. Many citizens, seeing the women flying |
| 134 | + toward the High Street, leaving their children crying at the open doors, |
| 135 | + hastened to don the cuirass, and supporting their somewhat uncertain |
| 136 | + courage with a musket or a partisan, directed their steps toward the |
| 137 | + hostelry of the Jolly Miller, before which was gathered, increasing |
| 138 | + every minute, a compact group, vociferous and full of curiosity.'; |
| 139 | +
|
| 140 | +set Words['3musk002']['book'] = 'The Three Musketeers'; |
| 141 | +set Words['3musk002']['para'] = 'In those times panics were common, and few days passed without |
| 142 | + some city or other registering in its archives an event of this kind. There were |
| 143 | + nobles, who made war against each other; there was the king, who made |
| 144 | + war against the cardinal; there was Spain, which made war against the |
| 145 | + king. Then, in addition to these concealed or public, secret or open |
| 146 | + wars, there were robbers, mendicants, Huguenots, wolves, and scoundrels, |
| 147 | + who made war upon everybody. The citizens always took up arms readily |
| 148 | + against thieves, wolves or scoundrels, often against nobles or |
| 149 | + Huguenots, sometimes against the king, but never against cardinal or |
| 150 | + Spain. It resulted, then, from this habit that on the said first Monday |
| 151 | + of April, 1625, the citizens, on hearing the clamor, and seeing neither |
| 152 | + the red-and-yellow standard nor the livery of the Duc de Richelieu, |
| 153 | + rushed toward the hostel of the Jolly Miller. When arrived there, the |
| 154 | + cause of the hubbub was apparent to all'; |
| 155 | +
|
| 156 | +set Words['3musk003']['book'] = 'The Three Musketeers'; |
| 157 | +set Words['3musk003']['para'] = 'You ought, I say, then, to husband the means you have, however |
| 158 | + large the sum may be; but you ought also to endeavor to perfect yourself in |
| 159 | + the exercises becoming a gentleman. I will write a letter today to the |
| 160 | + Director of the Royal Academy, and tomorrow he will admit you without |
| 161 | + any expense to yourself. Do not refuse this little service. Our |
| 162 | + best-born and richest gentlemen sometimes solicit it without being able |
| 163 | + to obtain it. You will learn horsemanship, swordsmanship in all its |
| 164 | + branches, and dancing. You will make some desirable acquaintances; and |
| 165 | + from time to time you can call upon me, just to tell me how you are |
| 166 | + getting on, and to say whether I can be of further service to you.'; |
| 167 | +
|
| 168 | +
|
| 169 | +set Words['thelostworld001']['book'] = 'The Lost World'; |
| 170 | +set Words['thelostworld001']['para'] = 'She sat with that proud, delicate profile of hers outlined |
| 171 | + against the red curtain. How beautiful she was! And yet how aloof! We had been |
| 172 | + friends, quite good friends; but never could I get beyond the same |
| 173 | + comradeship which I might have established with one of my |
| 174 | + fellow-reporters upon the Gazette,--perfectly frank, perfectly kindly, |
| 175 | + and perfectly unsexual. My instincts are all against a woman being too |
| 176 | + frank and at her ease with me. It is no compliment to a man. Where |
| 177 | + the real sex feeling begins, timidity and distrust are its companions, |
| 178 | + heritage from old wicked days when love and violence went often hand in |
| 179 | + hand. The bent head, the averted eye, the faltering voice, the wincing |
| 180 | + figure--these, and not the unshrinking gaze and frank reply, are the |
| 181 | + true signals of passion. Even in my short life I had learned as much |
| 182 | + as that--or had inherited it in that race memory which we call instinct.'; |
| 183 | +
|
| 184 | +set Words['thelostworld002']['book'] = 'The Lost World'; |
| 185 | +set Words['thelostworld002']['para'] = 'I always liked McArdle, the crabbed, old, round-backed, |
| 186 | + red-headed news editor, and I rather hoped that he liked me. Of course, Beaumont was |
| 187 | + the real boss; but he lived in the rarefied atmosphere of some Olympian |
| 188 | + height from which he could distinguish nothing smaller than an |
| 189 | + international crisis or a split in the Cabinet. Sometimes we saw him |
| 190 | + passing in lonely majesty to his inner sanctum, with his eyes staring |
| 191 | + vaguely and his mind hovering over the Balkans or the Persian Gulf. He |
| 192 | + was above and beyond us. But McArdle was his first lieutenant, and it |
| 193 | + was he that we knew. The old man nodded as I entered the room, and he |
| 194 | + pushed his spectacles far up on his bald forehead.'; |
| 195 | +
|
| 196 | +*/ |
0 commit comments