├── settings.gradle ├── gradle └── wrapper │ ├── gradle-wrapper.jar │ └── gradle-wrapper.properties ├── src └── main │ ├── resources │ ├── twitter-app.properties.template │ ├── log4j.properties │ └── kafkaStreamsTwitterTrainingData_clean.csv │ └── java │ └── bbejeck │ ├── processor │ ├── twitter │ │ └── TwitterClassificationProcessor.java │ ├── purchases │ │ ├── CustomerRewards.java │ │ ├── PurchasePatterns.java │ │ ├── CreditCardAnonymizer.java │ │ └── PurchaseProcessorDriver.java │ └── stocks │ │ ├── StockSummaryProcessor.java │ │ └── StockSummaryStatefulProcessorDriver.java │ ├── serializer │ ├── JsonSerializer.java │ └── JsonDeserializer.java │ ├── model │ ├── Tweet.java │ ├── StockTransactionCollector.java │ ├── StockTransactionSummary.java │ ├── RewardAccumulator.java │ ├── StockTransaction.java │ ├── PurchasePattern.java │ └── Purchase.java │ ├── streams │ ├── twitter │ │ ├── TwitterKStreamNLPDriver.java │ │ ├── HoseBirdTester.java │ │ ├── TwitterStreamsAnalyzer.java │ │ └── TwitterDataSource.java │ ├── RegexTest.java │ ├── purchases │ │ └── PurchaseKafkaStreamsDriver.java │ └── stocks │ │ └── StocksKafkaStreamsDriver.java │ └── nlp │ ├── LingPipeTester.java │ └── Classifier.java ├── bin └── create-topics.sh ├── streaming-workflows ├── purchases-config.json ├── stock-transactions-config.json ├── stock-transactions.json └── purchases.json ├── .gitignore ├── gradlew.bat ├── README.md ├── gradlew └── LICENSE /settings.gradle: -------------------------------------------------------------------------------- 1 | rootProject.name = 'kafka-streams' 2 | -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/bbejeck/kafka-streams/HEAD/gradle/wrapper/gradle-wrapper.jar -------------------------------------------------------------------------------- /src/main/resources/twitter-app.properties.template: -------------------------------------------------------------------------------- 1 | clientName= 2 | consumerKey= 3 | consumerSecret= 4 | token= 5 | tokenSecret= 6 | searchTerms= -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.properties: -------------------------------------------------------------------------------- 1 | #Sat Jul 16 21:02:57 EDT 2016 2 | distributionBase=GRADLE_USER_HOME 3 | distributionPath=wrapper/dists 4 | zipStoreBase=GRADLE_USER_HOME 5 | zipStorePath=wrapper/dists 6 | distributionUrl=https\://services.gradle.org/distributions/gradle-2.11-all.zip 7 | -------------------------------------------------------------------------------- /bin/create-topics.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | 4 | KAFKA_HOME=$1 5 | ZK_HOST=$2 6 | ZK_PORT=$3 7 | 8 | topics="src-topic patterns rewards purchases stocks stocks-out transaction-summary twitterData english french spanish" 9 | 10 | for topic in ${topics}; do 11 | echo "attempting to create topic ${topic}" 12 | ${KAFKA_HOME}/bin/kafka-topics.sh --create --topic ${topic} --partitions 1 --replication-factor 1 --zookeeper ${ZK_HOST}:${ZK_PORT} 13 | done 14 | -------------------------------------------------------------------------------- /streaming-workflows/purchases-config.json: -------------------------------------------------------------------------------- 1 | { 2 | "workflows": [ 3 | { 4 | "workflowName": "purchases", 5 | "workflowFilename": "purchases.json" 6 | } 7 | ], 8 | "producers": [ 9 | { 10 | "type": "kafka", 11 | "broker.server": "127.0.0.1", 12 | "broker.port": 9092, 13 | "topic": "src-topic", 14 | "flatten" : false, 15 | "sync": false 16 | }, 17 | { 18 | "type": "logger" 19 | } 20 | ] 21 | } -------------------------------------------------------------------------------- /streaming-workflows/stock-transactions-config.json: -------------------------------------------------------------------------------- 1 | { 2 | "workflows": [ 3 | { 4 | "workflowName": "stock-transactions", 5 | "workflowFilename": "stock-transactions.json" 6 | } 7 | ], 8 | "producers": [ 9 | { 10 | "type": "kafka", 11 | "broker.server": "127.0.0.1", 12 | "broker.port": 9092, 13 | "topic": "stocks", 14 | "flatten" :false, 15 | "sync": false 16 | }, 17 | { 18 | "type": "logger" 19 | } 20 | ] 21 | } -------------------------------------------------------------------------------- /streaming-workflows/stock-transactions.json: -------------------------------------------------------------------------------- 1 | { 2 | "eventFrequency": 400, 3 | "varyEventFrequency": true, 4 | "repeatWorkflow": true, 5 | "timeBetweenRepeat": 1500, 6 | "varyRepeatFrequency": true, 7 | "steps": [ 8 | { 9 | "config": [ 10 | { 11 | "timestamp": "now()", 12 | "symbol": "random('GOOG','MSFT','AAPL','YHOO','TWTR')", 13 | "amount": "double(100.0,60000.0)", 14 | "shares": "integer(100,1000)", 15 | "type": "random('purchase','sell')" 16 | } 17 | ], 18 | "duration": 0 19 | } 20 | ] 21 | } -------------------------------------------------------------------------------- /streaming-workflows/purchases.json: -------------------------------------------------------------------------------- 1 | { 2 | "eventFrequency": 400, 3 | "varyEventFrequency": true, 4 | "repeatWorkflow": true, 5 | "timeBetweenRepeat": 1500, 6 | "varyRepeatFrequency": true, 7 | "steps": [ 8 | { 9 | "config": [ 10 | { 11 | "lastName" : "lastName()", 12 | "firstName" : "firstName()", 13 | "creditCardNumber": "random('4929-3813-3266-4295', '5370-4638-8881-3020','4916-4811-5814-8111','4916-4034-9269-8783','5299-1561-5689-1938','5293-8502-0071-3058') ", 14 | "itemPurchased": "random('batteries','eggs','diapers','shampoo','shaving cream','doughnuts','beer')", 15 | "quantity": "integer(1,4)", 16 | "price": "double(3.95,14.99)", 17 | "purchaseDate": "date(\"2016/02/12T00:00:00\",\"2016/02/19T23:59:59\")" , 18 | "zipCode" : "random('20841','20852','19971','10005','21842')" 19 | } 20 | ], 21 | "duration": 0 22 | } 23 | ] 24 | } -------------------------------------------------------------------------------- /src/main/java/bbejeck/processor/twitter/TwitterClassificationProcessor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.processor.twitter; 18 | 19 | import bbejeck.model.Tweet; 20 | import org.apache.kafka.streams.processor.AbstractProcessor; 21 | 22 | /** 23 | * User: Bill Bejeck 24 | * Date: 4/21/16 25 | * Time: 9:10 PM 26 | */ 27 | public class TwitterClassificationProcessor extends AbstractProcessor { 28 | 29 | 30 | 31 | @Override 32 | public void process(String s, Tweet tweet) { 33 | 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/processor/purchases/CustomerRewards.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.processor.purchases; 18 | 19 | import bbejeck.model.Purchase; 20 | import bbejeck.model.RewardAccumulator; 21 | import org.apache.kafka.streams.processor.AbstractProcessor; 22 | 23 | /** 24 | * User: Bill Bejeck 25 | * Date: 2/20/16 26 | * Time: 9:44 AM 27 | */ 28 | public class CustomerRewards extends AbstractProcessor { 29 | 30 | @Override 31 | public void process(String key, Purchase value) { 32 | RewardAccumulator accumulator = RewardAccumulator.builder(value).build(); 33 | context().forward(key,accumulator); 34 | context().commit(); 35 | 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | ### Gradle template 2 | .gradle 3 | build/ 4 | 5 | # Ignore Gradle GUI config 6 | gradle-app.setting 7 | 8 | # Avoid ignoring Gradle wrapper jar file (.jar files are usually ignored) 9 | !gradle-wrapper.jar 10 | ### JetBrains template 11 | # Covers JetBrains IDEs: IntelliJ, RubyMine, PhpStorm, AppCode, PyCharm, CLion, Android Studio 12 | 13 | *.iml 14 | 15 | ## Directory-based project format: 16 | .idea/ 17 | # if you remove the above rule, at least ignore the following: 18 | 19 | # User-specific stuff: 20 | # .idea/workspace.xml 21 | # .idea/tasks.xml 22 | # .idea/dictionaries 23 | 24 | # Sensitive or high-churn files: 25 | # .idea/dataSources.ids 26 | # .idea/dataSources.xml 27 | # .idea/sqlDataSources.xml 28 | # .idea/dynamic.xml 29 | # .idea/uiDesigner.xml 30 | 31 | # Gradle: 32 | # .idea/gradle.xml 33 | # .idea/libraries 34 | 35 | # Mongo Explorer plugin: 36 | # .idea/mongoSettings.xml 37 | 38 | ## File-based project format: 39 | *.ipr 40 | *.iws 41 | 42 | ## Plugin-specific files: 43 | 44 | # IntelliJ 45 | /out/ 46 | 47 | # mpeltonen/sbt-idea plugin 48 | .idea_modules/ 49 | 50 | # JIRA plugin 51 | atlassian-ide-plugin.xml 52 | 53 | # Crashlytics plugin (for Android Studio and IntelliJ) 54 | com_crashlytics_export_strings.xml 55 | crashlytics.properties 56 | crashlytics-build.properties 57 | 58 | # Created by .ignore support plugin (hsz.mobi) 59 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/serializer/JsonSerializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.serializer; 18 | 19 | import com.google.gson.Gson; 20 | import org.apache.kafka.common.serialization.Serializer; 21 | 22 | import java.nio.charset.Charset; 23 | import java.util.Map; 24 | 25 | /** 26 | * User: Bill Bejeck 27 | * Date: 2/14/16 28 | * Time: 2:37 PM 29 | */ 30 | public class JsonSerializer implements Serializer { 31 | 32 | private Gson gson = new Gson(); 33 | 34 | @Override 35 | public void configure(Map map, boolean b) { 36 | 37 | } 38 | 39 | @Override 40 | public byte[] serialize(String topic, T t) { 41 | return gson.toJson(t).getBytes(Charset.forName("UTF-8")); 42 | } 43 | 44 | @Override 45 | public void close() { 46 | 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/processor/purchases/PurchasePatterns.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.processor.purchases; 18 | 19 | import bbejeck.model.Purchase; 20 | import bbejeck.model.PurchasePattern; 21 | import org.apache.kafka.streams.processor.AbstractProcessor; 22 | 23 | /** 24 | * User: Bill Bejeck 25 | * Date: 2/20/16 26 | * Time: 9:39 AM 27 | */ 28 | public class PurchasePatterns extends AbstractProcessor { 29 | 30 | @Override 31 | public void process(String key, Purchase value) { 32 | PurchasePattern purchasePattern = PurchasePattern.newBuilder().date(value.getPurchaseDate()) 33 | .item(value.getItemPurchased()) 34 | .zipCode(value.getZipCode()).build(); 35 | context().forward(key, purchasePattern); 36 | context().commit(); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/processor/purchases/CreditCardAnonymizer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.processor.purchases; 18 | 19 | import bbejeck.model.Purchase; 20 | import org.apache.kafka.streams.processor.AbstractProcessor; 21 | 22 | /** 23 | * User: Bill Bejeck 24 | * Date: 2/20/16 25 | * Time: 9:19 AM 26 | */ 27 | public class CreditCardAnonymizer extends AbstractProcessor { 28 | 29 | private static final String CC_NUMBER_REPLACEMENT="xxxx-xxxx-xxxx-"; 30 | 31 | @Override 32 | public void process(String key, Purchase purchase) { 33 | String last4Digits = purchase.getCreditCardNumber().split("-")[3]; 34 | Purchase updated = Purchase.builder(purchase).creditCardNumber(CC_NUMBER_REPLACEMENT+last4Digits).build(); 35 | context().forward(key,updated); 36 | context().commit(); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/model/Tweet.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.model; 18 | 19 | /** 20 | * User: Bill Bejeck 21 | * Date: 4/21/16 22 | * Time: 8:38 PM 23 | */ 24 | public class Tweet { 25 | 26 | private String id; 27 | private String text; 28 | private String language; 29 | 30 | public String getId() { 31 | return id; 32 | } 33 | 34 | public void setId(String id) { 35 | this.id = id; 36 | } 37 | 38 | public String getText() { 39 | return text; 40 | } 41 | 42 | public void setText(String text) { 43 | this.text = text; 44 | } 45 | 46 | public String getLanguage() { 47 | return language; 48 | } 49 | 50 | public void setLanguage(String language) { 51 | this.language = language; 52 | } 53 | 54 | 55 | @Override 56 | public String toString() { 57 | return "Tweet{" + 58 | "id='" + id + '\'' + 59 | ", text='" + text + '\'' + 60 | ", language='" + language + '\'' + 61 | '}'; 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/model/StockTransactionCollector.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.model; 18 | 19 | /** 20 | * User: Bill Bejeck 21 | * Date: 3/10/16 22 | * Time: 7:54 PM 23 | */ 24 | public class StockTransactionCollector { 25 | 26 | private double amount; 27 | private String tickerSymbol; 28 | private int sharesPurchased; 29 | private int sharesSold; 30 | 31 | public StockTransactionCollector add(StockTransaction transaction){ 32 | if(tickerSymbol == null){ 33 | tickerSymbol = transaction.getSymbol(); 34 | } 35 | 36 | this.amount += transaction.getAmount(); 37 | if(transaction.getType().equalsIgnoreCase("purchase")){ 38 | this.sharesPurchased += transaction.getShares(); 39 | } else{ 40 | this.sharesSold += transaction.getShares(); 41 | } 42 | return this; 43 | } 44 | 45 | @Override 46 | public String toString() { 47 | return "StockTransactionCollector{" + 48 | "amount=" + amount + 49 | ", tickerSymbol='" + tickerSymbol + '\'' + 50 | ", sharesPurchased=" + sharesPurchased + 51 | ", sharesSold=" + sharesSold + 52 | '}'; 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/serializer/JsonDeserializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.serializer; 18 | 19 | import com.google.gson.Gson; 20 | import org.apache.kafka.common.serialization.Deserializer; 21 | 22 | import java.util.Map; 23 | 24 | /** 25 | * User: Bill Bejeck 26 | * Date: 2/14/16 27 | * Time: 3:26 PM 28 | */ 29 | 30 | public class JsonDeserializer implements Deserializer { 31 | 32 | private Gson gson = new Gson(); 33 | private Class deserializedClass; 34 | 35 | public JsonDeserializer(Class deserializedClass) { 36 | this.deserializedClass = deserializedClass; 37 | } 38 | 39 | public JsonDeserializer() { 40 | } 41 | 42 | @Override 43 | @SuppressWarnings("unchecked") 44 | public void configure(Map map, boolean b) { 45 | if(deserializedClass == null) { 46 | deserializedClass = (Class) map.get("serializedClass"); 47 | } 48 | } 49 | 50 | @Override 51 | public T deserialize(String s, byte[] bytes) { 52 | if(bytes == null){ 53 | return null; 54 | } 55 | 56 | return gson.fromJson(new String(bytes),deserializedClass); 57 | 58 | } 59 | 60 | @Override 61 | public void close() { 62 | 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/model/StockTransactionSummary.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.model; 18 | 19 | /** 20 | * User: Bill Bejeck 21 | * Date: 2/6/16 22 | * Time: 3:32 PM 23 | */ 24 | public class StockTransactionSummary { 25 | 26 | public double amount; 27 | public String tickerSymbol; 28 | public int sharesPurchased; 29 | public int sharesSold; 30 | private long lastUpdatedTime; 31 | 32 | 33 | 34 | public void update(StockTransaction transaction){ 35 | this.amount += transaction.getAmount(); 36 | if(transaction.getType().equalsIgnoreCase("purchase")){ 37 | this.sharesPurchased += transaction.getShares(); 38 | } else{ 39 | this.sharesSold += transaction.getShares(); 40 | } 41 | this.lastUpdatedTime = System.currentTimeMillis(); 42 | } 43 | 44 | public boolean updatedWithinLastMillis(long currentTime, long limit){ 45 | return currentTime - this.lastUpdatedTime <= limit; 46 | } 47 | 48 | public static StockTransactionSummary fromTransaction(StockTransaction transaction){ 49 | StockTransactionSummary summary = new StockTransactionSummary(); 50 | summary.tickerSymbol = transaction.getSymbol(); 51 | summary.update(transaction); 52 | return summary; 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/streams/twitter/TwitterKStreamNLPDriver.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.streams.twitter; 18 | 19 | import java.util.concurrent.ExecutorService; 20 | import java.util.concurrent.Executors; 21 | 22 | /** 23 | * User: Bill Bejeck 24 | * Date: 4/27/16 25 | * Time: 8:46 PM 26 | */ 27 | public class TwitterKStreamNLPDriver { 28 | 29 | public static void main(String[] args) { 30 | 31 | System.out.println("Starting the KStreams Twitter analysis"); 32 | 33 | ExecutorService service = Executors.newFixedThreadPool(2); 34 | TwitterDataSource twitterDataSource = new TwitterDataSource(); 35 | TwitterStreamsAnalyzer twitterStreamsAnalyzer = new TwitterStreamsAnalyzer(); 36 | 37 | Runnable dataSourceRunner = twitterDataSource::run; 38 | Runnable streamsRunner = twitterStreamsAnalyzer::run; 39 | 40 | service.submit(dataSourceRunner); 41 | service.submit(streamsRunner); 42 | 43 | try { 44 | //run for 10 minutes 45 | Thread.sleep(600000); 46 | } catch (InterruptedException e) { 47 | Thread.currentThread().interrupt(); 48 | } finally { 49 | System.out.println("shutting down now"); 50 | service.shutdownNow(); 51 | twitterDataSource.stop(); 52 | twitterStreamsAnalyzer.stop(); 53 | System.out.println("all done...."); 54 | } 55 | 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2016 Bill Bejeck 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); 5 | # you may not use this file except in compliance with the License. 6 | # 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 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 19 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 20 | log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n 21 | 22 | log4j.appender.file=org.apache.log4j.RollingFileAppender 23 | log4j.appender.file.File=kafka_streams.log 24 | log4j.appender.file.MaxFileSize=10MB 25 | log4j.appender.file.MaxBackupIndex=2 26 | log4j.appender.file.layout=org.apache.log4j.PatternLayout 27 | log4j.appender.file.layout.ConversionPattern=[%d] %p %m (%c:%L)%n 28 | 29 | #log4j.logger.org.apache.kafka=INFO, file, stdout 30 | #log4j.logger.org.apache.kafka.clients.consumer.internals=INFO, file, stdout 31 | #log4j.logger.org.apache.kafka.common.metrics=INFO 32 | #log4j.logger.org.apache.kafka.streams.kstream.internals.KTableAggregate=INFO 33 | #log4j.logger.org.apache.kafka.streams.kstream=DEBUG 34 | #log4j.logger.org.apache.kafka.streams.processor.internals.StreamThread=INFO, stdout 35 | #log4j.logger.org.apache.kafka.streams.processor.internals.StreamTask=INFO, stdout 36 | #log4j.logger.org.apache.kafka.streams.processor.TopologyBuilder=INFO, stdout 37 | #log4j.logger.org.apache.kafka.streams.processor.internals.RecordQueue=INFO, stdout 38 | 39 | # zkclient can be verbose, during debugging it is common to adjust is separately 40 | #log4j.logger.org.I0Itec.zkclient.ZkClient=ERROR, file, stdout 41 | #log4j.logger.org.apache.zookeeper=ERROR, file, stdout 42 | 43 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/model/RewardAccumulator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.model; 18 | 19 | /** 20 | * User: Bill Bejeck 21 | * Date: 2/20/16 22 | * Time: 9:55 AM 23 | */ 24 | public class RewardAccumulator { 25 | 26 | private String customerName; 27 | private double purchaseTotal; 28 | 29 | private RewardAccumulator(String customerName, double purchaseTotal) { 30 | this.customerName = customerName; 31 | this.purchaseTotal = purchaseTotal; 32 | } 33 | 34 | public String getCustomerName() { 35 | return customerName; 36 | } 37 | 38 | public double getPurchaseTotal() { 39 | return purchaseTotal; 40 | } 41 | 42 | @Override 43 | public String toString() { 44 | return "RewardAccumulator{" + 45 | "customerName='" + customerName + '\'' + 46 | ", purchaseTotal=" + purchaseTotal + 47 | '}'; 48 | } 49 | 50 | public static Builder builder(Purchase purchase){return new Builder(purchase);} 51 | 52 | public static final class Builder { 53 | private String customerName; 54 | private double purchaseTotal; 55 | 56 | private Builder(Purchase purchase){ 57 | this.customerName = purchase.getLastName()+","+purchase.getFirstName(); 58 | this.purchaseTotal = purchase.getPrice() * purchase.getQuantity(); 59 | } 60 | 61 | 62 | public RewardAccumulator build(){ 63 | return new RewardAccumulator(customerName,purchaseTotal); 64 | } 65 | 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/model/StockTransaction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.model; 18 | 19 | import java.util.Date; 20 | 21 | /** 22 | * User: Bill Bejeck 23 | * Date: 2/14/16 24 | * Time: 2:31 PM 25 | */ 26 | public class StockTransaction { 27 | 28 | private String symbol; 29 | private String type; 30 | private double shares; 31 | private double amount; 32 | private Date timeStamp; 33 | 34 | 35 | public String getSymbol() { 36 | return symbol; 37 | } 38 | 39 | public void setSymbol(String symbol) { 40 | this.symbol = symbol; 41 | } 42 | 43 | public String getType() { 44 | return type; 45 | } 46 | 47 | public void setType(String type) { 48 | this.type = type; 49 | } 50 | 51 | public double getShares() { 52 | return shares; 53 | } 54 | 55 | public void setShares(double shares) { 56 | this.shares = shares; 57 | } 58 | 59 | public double getAmount() { 60 | return amount; 61 | } 62 | 63 | public void setAmount(double amount) { 64 | this.amount = amount; 65 | } 66 | 67 | public Date getTimeStamp() { 68 | return timeStamp; 69 | } 70 | 71 | public void setTimeStamp(Date timeStamp) { 72 | this.timeStamp = timeStamp; 73 | } 74 | 75 | @Override 76 | public String toString() { 77 | return "StockTransaction{" + 78 | "symbol='" + symbol + '\'' + 79 | ", type='" + type + '\'' + 80 | ", shares=" + shares + 81 | ", amount=" + amount + 82 | ", timeStamp=" + timeStamp + 83 | '}'; 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /gradlew.bat: -------------------------------------------------------------------------------- 1 | @if "%DEBUG%" == "" @echo off 2 | @rem ########################################################################## 3 | @rem 4 | @rem Gradle startup script for Windows 5 | @rem 6 | @rem ########################################################################## 7 | 8 | @rem Set local scope for the variables with windows NT shell 9 | if "%OS%"=="Windows_NT" setlocal 10 | 11 | @rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. 12 | set DEFAULT_JVM_OPTS= 13 | 14 | set DIRNAME=%~dp0 15 | if "%DIRNAME%" == "" set DIRNAME=. 16 | set APP_BASE_NAME=%~n0 17 | set APP_HOME=%DIRNAME% 18 | 19 | @rem Find java.exe 20 | if defined JAVA_HOME goto findJavaFromJavaHome 21 | 22 | set JAVA_EXE=java.exe 23 | %JAVA_EXE% -version >NUL 2>&1 24 | if "%ERRORLEVEL%" == "0" goto init 25 | 26 | echo. 27 | echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. 28 | echo. 29 | echo Please set the JAVA_HOME variable in your environment to match the 30 | echo location of your Java installation. 31 | 32 | goto fail 33 | 34 | :findJavaFromJavaHome 35 | set JAVA_HOME=%JAVA_HOME:"=% 36 | set JAVA_EXE=%JAVA_HOME%/bin/java.exe 37 | 38 | if exist "%JAVA_EXE%" goto init 39 | 40 | echo. 41 | echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% 42 | echo. 43 | echo Please set the JAVA_HOME variable in your environment to match the 44 | echo location of your Java installation. 45 | 46 | goto fail 47 | 48 | :init 49 | @rem Get command-line arguments, handling Windows variants 50 | 51 | if not "%OS%" == "Windows_NT" goto win9xME_args 52 | if "%@eval[2+2]" == "4" goto 4NT_args 53 | 54 | :win9xME_args 55 | @rem Slurp the command line arguments. 56 | set CMD_LINE_ARGS= 57 | set _SKIP=2 58 | 59 | :win9xME_args_slurp 60 | if "x%~1" == "x" goto execute 61 | 62 | set CMD_LINE_ARGS=%* 63 | goto execute 64 | 65 | :4NT_args 66 | @rem Get arguments from the 4NT Shell from JP Software 67 | set CMD_LINE_ARGS=%$ 68 | 69 | :execute 70 | @rem Setup the command line 71 | 72 | set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar 73 | 74 | @rem Execute Gradle 75 | "%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS% 76 | 77 | :end 78 | @rem End local scope for the variables with windows NT shell 79 | if "%ERRORLEVEL%"=="0" goto mainEnd 80 | 81 | :fail 82 | rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of 83 | rem the _cmd.exe /c_ return code! 84 | if not "" == "%GRADLE_EXIT_CONSOLE%" exit 1 85 | exit /b 1 86 | 87 | :mainEnd 88 | if "%OS%"=="Windows_NT" endlocal 89 | 90 | :omega 91 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/model/PurchasePattern.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.model; 18 | 19 | import java.util.Date; 20 | 21 | /** 22 | * User: Bill Bejeck 23 | * Date: 2/21/16 24 | * Time: 3:36 PM 25 | */ 26 | public class PurchasePattern { 27 | 28 | private String zipCode; 29 | private String item; 30 | private Date date; 31 | 32 | private PurchasePattern(Builder builder) { 33 | zipCode = builder.zipCode; 34 | item = builder.item; 35 | date = builder.date; 36 | } 37 | 38 | public static Builder newBuilder() { 39 | return new Builder(); 40 | } 41 | 42 | public static Builder builder(Purchase purchase){ 43 | return new Builder(purchase); 44 | 45 | } 46 | public String getZipCode() { 47 | return zipCode; 48 | } 49 | 50 | public String getItem() { 51 | return item; 52 | } 53 | 54 | public Date getDate() { 55 | return date; 56 | } 57 | 58 | 59 | @Override 60 | public String toString() { 61 | return "PurchasePattern{" + 62 | "zipCode='" + zipCode + '\'' + 63 | ", item='" + item + '\'' + 64 | ", date=" + date + 65 | '}'; 66 | } 67 | 68 | public static final class Builder { 69 | private String zipCode; 70 | private String item; 71 | private Date date; 72 | 73 | private Builder() { 74 | } 75 | 76 | private Builder(Purchase purchase) { 77 | this.zipCode = purchase.getZipCode(); 78 | this.item = purchase.getItemPurchased(); 79 | this.date = purchase.getPurchaseDate(); 80 | } 81 | 82 | public Builder zipCode(String val) { 83 | zipCode = val; 84 | return this; 85 | } 86 | 87 | public Builder item(String val) { 88 | item = val; 89 | return this; 90 | } 91 | 92 | public Builder date(Date val) { 93 | date = val; 94 | return this; 95 | } 96 | 97 | public PurchasePattern build() { 98 | return new PurchasePattern(this); 99 | } 100 | } 101 | } 102 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/processor/stocks/StockSummaryProcessor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.processor.stocks; 18 | 19 | import bbejeck.model.StockTransaction; 20 | import bbejeck.model.StockTransactionSummary; 21 | import org.apache.kafka.streams.processor.AbstractProcessor; 22 | import org.apache.kafka.streams.processor.ProcessorContext; 23 | import org.apache.kafka.streams.state.KeyValueIterator; 24 | import org.apache.kafka.streams.state.KeyValueStore; 25 | 26 | import java.util.Objects; 27 | 28 | /** 29 | * User: Bill Bejeck 30 | * Date: 1/25/16 31 | * Time: 8:13 PM 32 | */ 33 | 34 | @SuppressWarnings("unchecked") 35 | public class StockSummaryProcessor extends AbstractProcessor { 36 | 37 | private KeyValueStore summaryStore; 38 | private ProcessorContext context; 39 | 40 | 41 | public void process(String key, StockTransaction stockTransaction) { 42 | String currentSymbol = stockTransaction.getSymbol(); 43 | StockTransactionSummary transactionSummary = summaryStore.get(currentSymbol); 44 | if (transactionSummary == null) { 45 | transactionSummary = StockTransactionSummary.fromTransaction(stockTransaction); 46 | } else { 47 | transactionSummary.update(stockTransaction); 48 | } 49 | summaryStore.put(currentSymbol, transactionSummary); 50 | 51 | this.context.commit(); 52 | } 53 | 54 | 55 | @Override 56 | @SuppressWarnings("unchecked") 57 | public void init(ProcessorContext context) { 58 | this.context = context; 59 | this.context.schedule(10000); 60 | summaryStore = (KeyValueStore) this.context.getStateStore("stock-transactions"); 61 | Objects.requireNonNull(summaryStore, "State store can't be null"); 62 | 63 | } 64 | 65 | 66 | @Override 67 | public void punctuate(long streamTime) { 68 | KeyValueIterator it = summaryStore.all(); 69 | long currentTime = System.currentTimeMillis(); 70 | while (it.hasNext()) { 71 | StockTransactionSummary summary = it.next().value; 72 | if (summary.updatedWithinLastMillis(currentTime, 11000)) { 73 | this.context.forward(summary.tickerSymbol, summary); 74 | } 75 | } 76 | } 77 | 78 | } 79 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/streams/twitter/HoseBirdTester.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.streams.twitter; 18 | 19 | import com.google.common.collect.Lists; 20 | import com.twitter.hbc.ClientBuilder; 21 | import com.twitter.hbc.core.Client; 22 | import com.twitter.hbc.core.Constants; 23 | import com.twitter.hbc.core.Hosts; 24 | import com.twitter.hbc.core.HttpHosts; 25 | import com.twitter.hbc.core.endpoint.StatusesFilterEndpoint; 26 | import com.twitter.hbc.core.processor.StringDelimitedProcessor; 27 | import com.twitter.hbc.httpclient.auth.Authentication; 28 | import com.twitter.hbc.httpclient.auth.OAuth1; 29 | 30 | import java.util.List; 31 | import java.util.concurrent.BlockingQueue; 32 | import java.util.concurrent.LinkedBlockingDeque; 33 | 34 | /** 35 | * User: Bill Bejeck 36 | * Date: 3/28/16 37 | * Time: 8:55 PM 38 | */ 39 | public class HoseBirdTester { 40 | 41 | public static void main(String[] args) throws InterruptedException { 42 | 43 | BlockingQueue msgQueue = new LinkedBlockingDeque<>(); 44 | 45 | 46 | 47 | Hosts hosebirdHosts = new HttpHosts(Constants.STREAM_HOST); 48 | StatusesFilterEndpoint hosebirdEndpoint = new StatusesFilterEndpoint(); 49 | List terms = Lists.newArrayList("superman vs batman","#supermanvsbatman"); 50 | hosebirdEndpoint.trackTerms(terms); 51 | 52 | 53 | 54 | 55 | Authentication hosebirdAuth = new OAuth1("18qydWMuiUohwCtQpp1MOFCFr", 56 | "YrYhYd09LKZLbhsKT1o4XcEPl6HiAoNykiOxYBq0dAB8t0vRCo", 57 | "16972669-KSvyDEMc7dussPfW6a9Ru65L4eWGj637ciHLHZLyn", 58 | "ky53NE6cbBvtNLopto7o9gVyHDejSB2kPsRhHGKEd1MrS"); 59 | 60 | 61 | ClientBuilder clientBuilder = new ClientBuilder(); 62 | clientBuilder.name("bbejeck-hosebird") 63 | .hosts(hosebirdHosts) 64 | .authentication(hosebirdAuth) 65 | .endpoint(hosebirdEndpoint) 66 | .processor(new StringDelimitedProcessor(msgQueue)); 67 | 68 | Client hosebirdClient = clientBuilder.build(); 69 | hosebirdClient.connect(); 70 | 71 | for (int msgRead = 0; msgRead < 100; msgRead++) { 72 | String msg = msgQueue.take(); 73 | System.out.println(msg); 74 | } 75 | 76 | hosebirdClient.stop(); 77 | 78 | } 79 | } 80 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/nlp/LingPipeTester.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.nlp; 18 | 19 | import com.aliasi.classify.Classification; 20 | import com.aliasi.classify.Classified; 21 | import com.aliasi.classify.DynamicLMClassifier; 22 | import com.aliasi.lm.NGramBoundaryLM; 23 | 24 | import java.io.BufferedReader; 25 | import java.io.File; 26 | import java.io.FileInputStream; 27 | import java.io.InputStreamReader; 28 | import java.util.ArrayList; 29 | import java.util.HashSet; 30 | import java.util.List; 31 | import java.util.Set; 32 | 33 | /** 34 | * User: Bill Bejeck 35 | * Date: 4/9/16 36 | * Time: 6:24 PM 37 | */ 38 | public class LingPipeTester { 39 | 40 | public static void main(String[] args) throws Exception { 41 | File trainingData = new File("src/main/resources/kafkaStreamsTwitterTrainingData_clean.csv"); 42 | int maxCharNGram = 3; 43 | 44 | BufferedReader reader = new BufferedReader(new InputStreamReader(new FileInputStream(trainingData))); 45 | Set categorySet = new HashSet<>(); 46 | List annotatedData = new ArrayList<>(); 47 | String line = reader.readLine(); 48 | while (line !=null){ 49 | String[] data = line.split("#"); 50 | categorySet.add(data[0]); 51 | annotatedData.add(data); 52 | line = reader.readLine(); 53 | } 54 | System.out.println("read in all data"); 55 | reader.close(); 56 | String[] categories = categorySet.toArray(new String[0]); 57 | 58 | DynamicLMClassifier classifier 59 | = DynamicLMClassifier.createNGramBoundary(categories,maxCharNGram); 60 | for (String[] row: annotatedData) { 61 | String truth = row[0]; 62 | String text = row[1]; 63 | Classification classification = new Classification(truth); 64 | Classified classified = new Classified<>(text,classification); 65 | classifier.handle(classified); 66 | } 67 | System.out.println("training complete"); 68 | 69 | reader = new BufferedReader(new InputStreamReader(System.in)); 70 | 71 | System.out.println("enter text, I'll tell you the language"); 72 | String text; 73 | while (!(text = reader.readLine()).equalsIgnoreCase("quit")) { 74 | Classification classification = classifier.classify(text); 75 | System.out.println("Entered -> " + text); 76 | System.out.println("lang -> " + classification.bestCategory()); 77 | } 78 | } 79 | } 80 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/streams/RegexTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.streams; 18 | 19 | import org.apache.kafka.clients.consumer.ConsumerConfig; 20 | import org.apache.kafka.common.serialization.Serdes; 21 | import org.apache.kafka.streams.KafkaStreams; 22 | import org.apache.kafka.streams.StreamsConfig; 23 | import org.apache.kafka.streams.kstream.KStream; 24 | import org.apache.kafka.streams.kstream.KStreamBuilder; 25 | import org.apache.kafka.streams.processor.WallclockTimestampExtractor; 26 | 27 | import java.util.Properties; 28 | import java.util.regex.Pattern; 29 | 30 | /** 31 | * Example on Using Regex Patterns For Source Topics 32 | */ 33 | public class RegexTest { 34 | 35 | 36 | public static void main(String[] args) { 37 | 38 | StreamsConfig streamingConfig = new StreamsConfig(getProperties()); 39 | KStreamBuilder kStreamBuilder = new KStreamBuilder(); 40 | 41 | 42 | KStream patternStreamI = kStreamBuilder.stream(Serdes.String(), Serdes.String(), Pattern.compile("topic-\\d")); 43 | KStream namedTopicKStream = kStreamBuilder.stream(Serdes.String(), Serdes.String(), "topic-Z"); 44 | KStream patternStreamII = kStreamBuilder.stream(Serdes.String(), Serdes.String(), Pattern.compile("topic-[A-Y]+")); 45 | 46 | patternStreamI.print("pattern-\\d"); 47 | namedTopicKStream.print("topic-Z"); 48 | patternStreamII.print("topic-[A-Y]+"); 49 | 50 | 51 | System.out.println("Starting stream regex consumer Example"); 52 | KafkaStreams kafkaStreams = new KafkaStreams(kStreamBuilder, streamingConfig); 53 | kafkaStreams.start(); 54 | 55 | 56 | } 57 | 58 | 59 | private static Properties getProperties() { 60 | Properties props = new Properties(); 61 | props.put(StreamsConfig.APPLICATION_ID_CONFIG, "regex_id"); 62 | props.put(ConsumerConfig.GROUP_ID_CONFIG, "regex_group_id"); 63 | props.put(ConsumerConfig.CLIENT_ID_CONFIG, "regex_id"); 64 | props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); 65 | props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, "5000"); 66 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); 67 | props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); 68 | props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, "2"); 69 | props.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "15000"); 70 | props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); 71 | props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); 72 | props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 1); 73 | props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class); 74 | return props; 75 | } 76 | } 77 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # kafka-streams 2 | This is the repository for the examples of using Kafka streams covered in the blog posts: 3 | 4 | * [Kafka Streams - The Processor API](http://codingjunkie.net/kafka-processor-part1/) 5 | * [Kafka Streams - The KStreams API](http://codingjunkie.net/kafka-streams-part2/) 6 | * [Machine Learning with Kafka Streams](http://codingjunkie.net/kafka-streams-machine-learning/) 7 | 8 | 9 | ## Requirements to build this project 10 | 11 | 1. Java 8 12 | 2. Gradle 13 | 14 | ## Requirements to run the examples 15 | 16 | 1. [kafka](https://github.com/apache/kafka) version kafka_2.11-0.10.1.0 see the section marked "Running a task on a particular version of Scala" 17 | 2. The [json-data-generator](https://github.com/acesinc/json-data-generator) from [ACES,Inc](http://acesinc.net/) 18 | 19 | 20 | ## Setup Instructions 21 | 22 | #### Extact the kafka_2.11-0.10.1.0.tgz file #### 23 | tar -xvzf kafka_2.11-0.10.1.0.tgz 24 | 25 | 26 | #### Start zookeeper and kafka 27 | ``` 28 | kafka-install-dir/bin/zookeeper-server-start.sh kafka-install-dir/conf/zookeeper.properties 29 | kafka-install-dir/bin/kafka-server-start.sh kafka-install-dir/conf/server.properties 30 | ``` 31 | 32 | #### Install the Json-Data-Generator 33 | Download the latest [json-data-generator release](https://github.com/acesinc/json-data-generator/releases) and follow the install instructions [here](http://acesinc.net/introducing-a-streaming-json-data-generator/) 34 | 35 | #### Setup the kafka-streams repo 36 | Clone or fork the repo 37 | ``` 38 | git clone git@github.com:bbejeck/kafka-streams 39 | cd kafka-streams 40 | ``` 41 | Then copy the json config files to json generator conf directory 42 | ``` 43 | cp streaming-workflows/* /json-data-generator-1.2.0/conf 44 | ``` 45 | 46 | Create all the topics required by the examples 47 | ``` 48 | ./bin/create-topics.sh /usr/local/kafka_2.11-0.10.1.0 localhost 2181 49 | args are kafka home, zookeeper host and zookeeper port adjust accordingly 50 | ``` 51 | 52 | ### Running the Purchase Processor API KStreams API Examples ### 53 | cd /json-data-generator-1.2.0/ 54 | java -jar json-data-generator-1.2.0 purchases-config.json 55 | cd kafka-streams 56 | ./gradlew runPurchaseProcessor | runPurchaseStreams 57 | 58 | 59 | ### Running the Stock Trades Processor API or KStreams API Examples ### 60 | cd /json-data-generator-1.2.0/ 61 | java -jar json-data-generator-1.2.0 stock-transactions-config.json 62 | cd kafka-streams 63 | ./gradlew runStockProcessor | runStockStreams 64 | 65 | ### Running the Twitter KStreams Language Classification Example ### 66 | rename src/main/resources/twitter-app.properties.template to twitter-app.properties 67 | fill out the properties file with all the required values 68 | 69 | cd kafka-streams 70 | ./gradlew runTwitterKstreamNLP 71 | 72 | ### Viewing the results of the purchase streaming examples ### 73 | cd kafka_install-dir/bin 74 | ./kafka-console-consumer --topic [patterns|rewards|purchases] --zookeeper localhost:2181 75 | 76 | ### Viewing the results of the stock-trading streaming examples ### 77 | cd kafka_install-dir/bin 78 | ./kafka-console-consumer --topic [stocks-out|transaction-summary] --zookeeper localhost:2181 79 | 80 | ### Viewing the results of the Twitter KStreams Language Classification Example ### 81 | cd kafka_install-dir/bin 82 | ./kafka-console-consumer --topic [english|french|spanish] --zookeeper localhost:2181 83 | 84 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/nlp/Classifier.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.nlp; 18 | 19 | import com.aliasi.classify.Classification; 20 | import com.aliasi.classify.Classified; 21 | import com.aliasi.classify.DynamicLMClassifier; 22 | import com.aliasi.lm.NGramBoundaryLM; 23 | 24 | import java.io.BufferedReader; 25 | import java.io.File; 26 | import java.io.FileInputStream; 27 | import java.io.IOException; 28 | import java.io.InputStreamReader; 29 | import java.util.ArrayList; 30 | import java.util.HashSet; 31 | import java.util.List; 32 | import java.util.Set; 33 | 34 | /** 35 | * User: Bill Bejeck 36 | * Date: 4/20/16 37 | * Time: 10:01 PM 38 | */ 39 | public class Classifier { 40 | 41 | private DynamicLMClassifier classifier; 42 | private int maxCharNGram = 3; 43 | private String trainingDataDelimiter; 44 | 45 | public Classifier(String trainingDataDelimiter) { 46 | this.trainingDataDelimiter = trainingDataDelimiter; 47 | } 48 | 49 | public Classifier(){ 50 | this("#"); 51 | } 52 | 53 | public void train(File trainingData) { 54 | Set categorySet = new HashSet<>(); 55 | List annotatedData = new ArrayList<>(); 56 | fillCategoriesAndAnnotatedData(trainingData, categorySet, annotatedData); 57 | trainClassifier(categorySet, annotatedData); 58 | } 59 | 60 | private void trainClassifier(Set categorySet, List annotatedData){ 61 | String[] categories = categorySet.toArray(new String[0]); 62 | classifier = DynamicLMClassifier.createNGramBoundary(categories,maxCharNGram); 63 | for (String[] row: annotatedData) { 64 | String actualClassification = row[0]; 65 | String text = row[1]; 66 | Classification classification = new Classification(actualClassification); 67 | Classified classified = new Classified<>(text,classification); 68 | classifier.handle(classified); 69 | } 70 | } 71 | 72 | 73 | private void fillCategoriesAndAnnotatedData(File trainingData, 74 | Set categorySet, 75 | List annotatedData) { 76 | try (BufferedReader reader = new BufferedReader(new InputStreamReader(new FileInputStream(trainingData)))) { 77 | 78 | String line = reader.readLine(); 79 | while (line != null) { 80 | String[] data = line.split(trainingDataDelimiter); 81 | categorySet.add(data[0]); 82 | annotatedData.add(data); 83 | line = reader.readLine(); 84 | } 85 | 86 | } catch (IOException e){ 87 | throw new RuntimeException(e); 88 | } 89 | } 90 | 91 | 92 | public String classify(String text){ 93 | return classifier.classify(text.trim()).bestCategory().toLowerCase(); 94 | } 95 | 96 | } 97 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/processor/purchases/PurchaseProcessorDriver.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.processor.purchases; 18 | 19 | import bbejeck.model.Purchase; 20 | import bbejeck.model.PurchasePattern; 21 | import bbejeck.model.RewardAccumulator; 22 | import bbejeck.serializer.JsonDeserializer; 23 | import bbejeck.serializer.JsonSerializer; 24 | import org.apache.kafka.common.serialization.StringDeserializer; 25 | import org.apache.kafka.common.serialization.StringSerializer; 26 | import org.apache.kafka.streams.KafkaStreams; 27 | import org.apache.kafka.streams.StreamsConfig; 28 | import org.apache.kafka.streams.processor.TopologyBuilder; 29 | import org.apache.kafka.streams.processor.WallclockTimestampExtractor; 30 | 31 | import java.util.Properties; 32 | 33 | /** 34 | * User: Bill Bejeck 35 | * Date: 11/5/15 36 | * Time: 10:22 PM 37 | */ 38 | public class PurchaseProcessorDriver { 39 | 40 | public static void main(String[] args) throws Exception { 41 | 42 | StreamsConfig streamingConfig = new StreamsConfig(getProperties()); 43 | 44 | JsonDeserializer purchaseJsonDeserializer = new JsonDeserializer<>(Purchase.class); 45 | JsonSerializer purchaseJsonSerializer = new JsonSerializer<>(); 46 | JsonSerializer rewardAccumulatorJsonSerializer = new JsonSerializer<>(); 47 | JsonSerializer purchasePatternJsonSerializer = new JsonSerializer<>(); 48 | 49 | StringDeserializer stringDeserializer = new StringDeserializer(); 50 | StringSerializer stringSerializer = new StringSerializer(); 51 | 52 | TopologyBuilder topologyBuilder = new TopologyBuilder(); 53 | topologyBuilder.addSource("SOURCE", stringDeserializer, purchaseJsonDeserializer, "src-topic") 54 | 55 | .addProcessor("PROCESS", CreditCardAnonymizer::new, "SOURCE") 56 | .addProcessor("PROCESS2", PurchasePatterns::new, "PROCESS") 57 | .addProcessor("PROCESS3", CustomerRewards::new, "PROCESS") 58 | 59 | .addSink("SINK", "patterns", stringSerializer, purchasePatternJsonSerializer, "PROCESS2") 60 | .addSink("SINK2", "rewards",stringSerializer, rewardAccumulatorJsonSerializer, "PROCESS3") 61 | .addSink("SINK3", "purchases", stringSerializer, purchaseJsonSerializer, "PROCESS"); 62 | 63 | System.out.println("Starting PurchaseProcessor Example"); 64 | KafkaStreams streaming = new KafkaStreams(topologyBuilder, streamingConfig); 65 | streaming.start(); 66 | System.out.println("Now started PurchaseProcessor Example"); 67 | 68 | } 69 | 70 | private static Properties getProperties() { 71 | Properties props = new Properties(); 72 | props.put(StreamsConfig.CLIENT_ID_CONFIG, "Example-Processor-Job"); 73 | props.put("group.id", "test-consumer-group"); 74 | props.put(StreamsConfig.APPLICATION_ID_CONFIG, "testing-processor-api"); 75 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); 76 | props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); 77 | props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 1); 78 | props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class); 79 | return props; 80 | } 81 | } -------------------------------------------------------------------------------- /src/main/java/bbejeck/processor/stocks/StockSummaryStatefulProcessorDriver.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.processor.stocks; 18 | 19 | import bbejeck.model.StockTransaction; 20 | import bbejeck.model.StockTransactionSummary; 21 | import bbejeck.serializer.JsonDeserializer; 22 | import bbejeck.serializer.JsonSerializer; 23 | import org.apache.kafka.common.serialization.Serde; 24 | import org.apache.kafka.common.serialization.Serdes; 25 | import org.apache.kafka.common.serialization.StringDeserializer; 26 | import org.apache.kafka.common.serialization.StringSerializer; 27 | import org.apache.kafka.streams.KafkaStreams; 28 | import org.apache.kafka.streams.StreamsConfig; 29 | import org.apache.kafka.streams.processor.TopologyBuilder; 30 | import org.apache.kafka.streams.processor.WallclockTimestampExtractor; 31 | import org.apache.kafka.streams.state.Stores; 32 | 33 | import java.util.Properties; 34 | 35 | /** 36 | * User: Bill Bejeck 37 | * Date: 2/8/16 38 | * Time: 5:11 PM 39 | */ 40 | public class StockSummaryStatefulProcessorDriver { 41 | 42 | public static void main(String[] args) { 43 | 44 | StreamsConfig streamingConfig = new StreamsConfig(getProperties()); 45 | 46 | TopologyBuilder builder = new TopologyBuilder(); 47 | 48 | JsonSerializer stockTxnSummarySerializer = new JsonSerializer<>(); 49 | JsonDeserializer stockTxnSummaryDeserializer = new JsonDeserializer<>(StockTransactionSummary.class); 50 | JsonDeserializer stockTxnDeserializer = new JsonDeserializer<>(StockTransaction.class); 51 | JsonSerializer stockTxnJsonSerializer = new JsonSerializer<>(); 52 | StringSerializer stringSerializer = new StringSerializer(); 53 | StringDeserializer stringDeserializer = new StringDeserializer(); 54 | 55 | Serde stockTransactionSummarySerde = Serdes.serdeFrom(stockTxnSummarySerializer,stockTxnSummaryDeserializer); 56 | 57 | builder.addSource("stocks-source", stringDeserializer, stockTxnDeserializer, "stocks") 58 | .addProcessor("summary", StockSummaryProcessor::new, "stocks-source") 59 | .addStateStore(Stores.create("stock-transactions").withStringKeys() 60 | .withValues(stockTransactionSummarySerde).inMemory().maxEntries(100).build(),"summary") 61 | .addSink("sink", "stocks-out", stringSerializer,stockTxnJsonSerializer,"stocks-source") 62 | .addSink("sink-2", "transaction-summary", stringSerializer, stockTxnSummarySerializer, "summary"); 63 | 64 | System.out.println("Starting StockSummaryStatefulProcessor Example"); 65 | KafkaStreams streaming = new KafkaStreams(builder, streamingConfig); 66 | streaming.start(); 67 | System.out.println("StockSummaryStatefulProcessor Example now started"); 68 | 69 | } 70 | 71 | private static Properties getProperties() { 72 | Properties props = new Properties(); 73 | props.put(StreamsConfig.CLIENT_ID_CONFIG, "Sample-Stateful-Processor"); 74 | props.put("group.id", "test-consumer-group"); 75 | props.put(StreamsConfig.APPLICATION_ID_CONFIG, "stateful_processor_id"); 76 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); 77 | props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); 78 | props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 1); 79 | props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class); 80 | return props; 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/streams/twitter/TwitterStreamsAnalyzer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.streams.twitter; 18 | 19 | import bbejeck.model.Tweet; 20 | import bbejeck.nlp.Classifier; 21 | import bbejeck.serializer.JsonDeserializer; 22 | import bbejeck.serializer.JsonSerializer; 23 | import org.apache.commons.lang3.StringUtils; 24 | import org.apache.kafka.common.serialization.Serde; 25 | import org.apache.kafka.common.serialization.Serdes; 26 | import org.apache.kafka.streams.KafkaStreams; 27 | import org.apache.kafka.streams.StreamsConfig; 28 | import org.apache.kafka.streams.kstream.KStream; 29 | import org.apache.kafka.streams.kstream.KStreamBuilder; 30 | import org.apache.kafka.streams.kstream.KeyValueMapper; 31 | import org.apache.kafka.streams.kstream.Predicate; 32 | import org.apache.kafka.streams.processor.WallclockTimestampExtractor; 33 | 34 | import java.io.File; 35 | import java.io.IOException; 36 | import java.util.Properties; 37 | 38 | /** 39 | * User: Bill Bejeck 40 | * Date: 4/21/16 41 | * Time: 8:51 PM 42 | */ 43 | @SuppressWarnings("unchecked") 44 | public class TwitterStreamsAnalyzer { 45 | 46 | private KafkaStreams kafkaStreams; 47 | 48 | public static void main(String[] args) throws IOException { 49 | TwitterStreamsAnalyzer streamsAnalyzer = new TwitterStreamsAnalyzer(); 50 | streamsAnalyzer.run(); 51 | } 52 | 53 | public void run() { 54 | StreamsConfig streamsConfig = new StreamsConfig(getProperties()); 55 | 56 | JsonSerializer tweetJsonSerializer = new JsonSerializer<>(); 57 | JsonDeserializer tweetJsonDeserializer = new JsonDeserializer<>(Tweet.class); 58 | Serde tweetSerde = Serdes.serdeFrom(tweetJsonSerializer, tweetJsonDeserializer); 59 | 60 | KStreamBuilder kStreamBuilder = new KStreamBuilder(); 61 | 62 | Classifier classifier = new Classifier(); 63 | classifier.train(new File("src/main/resources/kafkaStreamsTwitterTrainingData_clean.csv")); 64 | 65 | KeyValueMapper languageToKey = (k, v) -> 66 | StringUtils.isNotBlank(v.getText()) ? classifier.classify(v.getText()):"unknown"; 67 | 68 | Predicate isEnglish = (k, v) -> k.equals("english"); 69 | Predicate isFrench = (k, v) -> k.equals("french"); 70 | Predicate isSpanish = (k, v) -> k.equals("spanish"); 71 | 72 | KStream tweetKStream = kStreamBuilder.stream(Serdes.String(), tweetSerde, "twitterData"); 73 | 74 | KStream[] filteredStreams = tweetKStream.selectKey(languageToKey).branch(isEnglish, isFrench, isSpanish); 75 | 76 | filteredStreams[0].to(Serdes.String(), tweetSerde, "english"); 77 | filteredStreams[1].to(Serdes.String(), tweetSerde, "french"); 78 | filteredStreams[2].to(Serdes.String(), tweetSerde, "spanish"); 79 | 80 | kafkaStreams = new KafkaStreams(kStreamBuilder, streamsConfig); 81 | System.out.println("Starting twitter analysis streams"); 82 | kafkaStreams.start(); 83 | System.out.println("Started"); 84 | 85 | } 86 | 87 | public void stop() { 88 | kafkaStreams.close(); 89 | System.out.println("KStreams stopped"); 90 | } 91 | 92 | 93 | private static Properties getProperties() { 94 | Properties props = new Properties(); 95 | props.put(StreamsConfig.CLIENT_ID_CONFIG, "Twitter-Streams-Analysis"); 96 | props.put("group.id", "twitter-streams"); 97 | props.put(StreamsConfig.APPLICATION_ID_CONFIG, "twitter-streams-id"); 98 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); 99 | props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); 100 | props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 1); 101 | props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class); 102 | return props; 103 | } 104 | } 105 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/streams/purchases/PurchaseKafkaStreamsDriver.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.streams.purchases; 18 | 19 | import bbejeck.model.Purchase; 20 | import bbejeck.model.PurchasePattern; 21 | import bbejeck.model.RewardAccumulator; 22 | import bbejeck.serializer.JsonDeserializer; 23 | import bbejeck.serializer.JsonSerializer; 24 | import org.apache.kafka.common.serialization.Serde; 25 | import org.apache.kafka.common.serialization.Serdes; 26 | import org.apache.kafka.common.serialization.StringDeserializer; 27 | import org.apache.kafka.common.serialization.StringSerializer; 28 | import org.apache.kafka.streams.KafkaStreams; 29 | import org.apache.kafka.streams.StreamsConfig; 30 | import org.apache.kafka.streams.kstream.KStream; 31 | import org.apache.kafka.streams.kstream.KStreamBuilder; 32 | import org.apache.kafka.streams.processor.WallclockTimestampExtractor; 33 | 34 | import java.util.Properties; 35 | 36 | /** 37 | * User: Bill Bejeck 38 | * Date: 3/7/16 39 | * Time: 5:34 PM 40 | */ 41 | public class PurchaseKafkaStreamsDriver { 42 | 43 | public static void main(String[] args) { 44 | 45 | 46 | StreamsConfig streamsConfig = new StreamsConfig(getProperties()); 47 | 48 | JsonDeserializer purchaseJsonDeserializer = new JsonDeserializer<>(Purchase.class); 49 | JsonSerializer purchaseJsonSerializer = new JsonSerializer<>(); 50 | 51 | JsonSerializer rewardAccumulatorJsonSerializer = new JsonSerializer<>(); 52 | JsonDeserializer rewardAccumulatorJsonDeserializer = new JsonDeserializer<>(RewardAccumulator.class); 53 | 54 | Serde rewardAccumulatorSerde = Serdes.serdeFrom(rewardAccumulatorJsonSerializer,rewardAccumulatorJsonDeserializer); 55 | 56 | JsonSerializer purchasePatternJsonSerializer = new JsonSerializer<>(); 57 | JsonDeserializer purchasePatternJsonDeserializer = new JsonDeserializer<>(PurchasePattern.class); 58 | 59 | Serde purchasePatternSerde = Serdes.serdeFrom(purchasePatternJsonSerializer,purchasePatternJsonDeserializer); 60 | 61 | Serde purchaseSerde = Serdes.serdeFrom(purchaseJsonSerializer,purchaseJsonDeserializer); 62 | 63 | Serde stringSerde = Serdes.String(); 64 | 65 | KStreamBuilder kStreamBuilder = new KStreamBuilder(); 66 | 67 | 68 | KStream purchaseKStream = kStreamBuilder.stream(stringSerde,purchaseSerde,"src-topic") 69 | .mapValues(p -> Purchase.builder(p).maskCreditCard().build()); 70 | 71 | purchaseKStream.mapValues(purchase -> PurchasePattern.builder(purchase).build()).to(stringSerde,purchasePatternSerde,"patterns"); 72 | 73 | purchaseKStream.mapValues(purchase -> RewardAccumulator.builder(purchase).build()).to(stringSerde,rewardAccumulatorSerde,"rewards"); 74 | 75 | purchaseKStream.to(stringSerde,purchaseSerde,"purchases"); 76 | 77 | System.out.println("Starting PurchaseStreams Example"); 78 | KafkaStreams kafkaStreams = new KafkaStreams(kStreamBuilder,streamsConfig); 79 | kafkaStreams.start(); 80 | System.out.println("Now started PurchaseStreams Example"); 81 | 82 | } 83 | 84 | 85 | 86 | 87 | private static Properties getProperties() { 88 | Properties props = new Properties(); 89 | props.put(StreamsConfig.CLIENT_ID_CONFIG, "Example-Kafka-Streams-Job"); 90 | props.put("group.id", "streams-purchases"); 91 | props.put(StreamsConfig.APPLICATION_ID_CONFIG, "testing-streams-api"); 92 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); 93 | props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); 94 | props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 1); 95 | props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class); 96 | return props; 97 | } 98 | 99 | } 100 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/streams/stocks/StocksKafkaStreamsDriver.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.streams.stocks; 18 | 19 | import bbejeck.model.StockTransaction; 20 | import bbejeck.model.StockTransactionCollector; 21 | import bbejeck.serializer.JsonDeserializer; 22 | import bbejeck.serializer.JsonSerializer; 23 | import org.apache.kafka.common.serialization.Serde; 24 | import org.apache.kafka.common.serialization.Serdes; 25 | import org.apache.kafka.common.serialization.StringDeserializer; 26 | import org.apache.kafka.common.serialization.StringSerializer; 27 | import org.apache.kafka.streams.KafkaStreams; 28 | import org.apache.kafka.streams.KeyValue; 29 | import org.apache.kafka.streams.StreamsConfig; 30 | import org.apache.kafka.streams.kstream.KStream; 31 | import org.apache.kafka.streams.kstream.KStreamBuilder; 32 | import org.apache.kafka.streams.kstream.TimeWindows; 33 | import org.apache.kafka.streams.kstream.Windowed; 34 | import org.apache.kafka.streams.kstream.internals.WindowedDeserializer; 35 | import org.apache.kafka.streams.kstream.internals.WindowedSerializer; 36 | import org.apache.kafka.streams.processor.WallclockTimestampExtractor; 37 | 38 | import java.util.Properties; 39 | 40 | /** 41 | * User: Bill Bejeck 42 | * Date: 3/9/16 43 | * Time: 9:21 PM 44 | */ 45 | public class StocksKafkaStreamsDriver { 46 | 47 | public static void main(String[] args) { 48 | 49 | StreamsConfig streamingConfig = new StreamsConfig(getProperties()); 50 | 51 | JsonSerializer stockTransactionsSerializer = new JsonSerializer<>(); 52 | JsonDeserializer stockTransactionsDeserializer = new JsonDeserializer<>(StockTransactionCollector.class); 53 | JsonDeserializer stockTxnDeserializer = new JsonDeserializer<>(StockTransaction.class); 54 | JsonSerializer stockTxnJsonSerializer = new JsonSerializer<>(); 55 | Serde transactionSerde = Serdes.serdeFrom(stockTxnJsonSerializer,stockTxnDeserializer); 56 | StringSerializer stringSerializer = new StringSerializer(); 57 | StringDeserializer stringDeserializer = new StringDeserializer(); 58 | Serde stringSerde = Serdes.serdeFrom(stringSerializer,stringDeserializer); 59 | Serde collectorSerde = Serdes.serdeFrom(stockTransactionsSerializer,stockTransactionsDeserializer); 60 | WindowedSerializer windowedSerializer = new WindowedSerializer<>(stringSerializer); 61 | WindowedDeserializer windowedDeserializer = new WindowedDeserializer<>(stringDeserializer); 62 | Serde> windowedSerde = Serdes.serdeFrom(windowedSerializer,windowedDeserializer); 63 | 64 | KStreamBuilder kStreamBuilder = new KStreamBuilder(); 65 | 66 | 67 | KStream transactionKStream = kStreamBuilder.stream(stringSerde,transactionSerde,"stocks"); 68 | 69 | transactionKStream.map((k,v)-> new KeyValue<>(v.getSymbol(),v)) 70 | .through(stringSerde, transactionSerde,"stocks-out") 71 | .groupBy((k,v) -> k, stringSerde, transactionSerde) 72 | .aggregate(StockTransactionCollector::new, 73 | (k, v, stockTransactionCollector) -> stockTransactionCollector.add(v), 74 | TimeWindows.of(10000), 75 | collectorSerde, "stock-summaries") 76 | .to(windowedSerde,collectorSerde,"transaction-summary"); 77 | 78 | 79 | System.out.println("Starting StockStreams Example"); 80 | KafkaStreams kafkaStreams = new KafkaStreams(kStreamBuilder,streamingConfig); 81 | kafkaStreams.start(); 82 | System.out.println("Now started StockStreams Example"); 83 | 84 | } 85 | 86 | private static Properties getProperties() { 87 | Properties props = new Properties(); 88 | props.put(StreamsConfig.CLIENT_ID_CONFIG, "Stocks-Streams-Processor"); 89 | props.put("group.id", "stock-streams"); 90 | props.put(StreamsConfig.APPLICATION_ID_CONFIG, "stocks_streams_id"); 91 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); 92 | props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); 93 | props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 1); 94 | props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class); 95 | return props; 96 | } 97 | } 98 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/model/Purchase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.model; 18 | 19 | import java.util.Date; 20 | import java.util.Objects; 21 | 22 | /** 23 | * User: Bill Bejeck 24 | * Date: 2/20/16 25 | * Time: 9:09 AM 26 | */ 27 | public class Purchase { 28 | 29 | private String firstName; 30 | private String lastName; 31 | private String creditCardNumber; 32 | private String itemPurchased; 33 | int quantity; 34 | double price; 35 | private Date purchaseDate; 36 | private String zipCode; 37 | 38 | private Purchase(Builder builder) { 39 | firstName = builder.firstName; 40 | lastName = builder.lastName; 41 | creditCardNumber = builder.creditCardNumber; 42 | itemPurchased = builder.itemPurchased; 43 | quantity = builder.quanity; 44 | price = builder.price; 45 | purchaseDate = builder.purchaseDate; 46 | zipCode = builder.zipCode; 47 | } 48 | 49 | public static Builder builder() { 50 | return new Builder(); 51 | } 52 | 53 | public static Builder builder(Purchase copy) { 54 | Builder builder = new Builder(); 55 | builder.firstName = copy.firstName; 56 | builder.lastName = copy.lastName; 57 | builder.creditCardNumber = copy.creditCardNumber; 58 | builder.itemPurchased = copy.itemPurchased; 59 | builder.quanity = copy.quantity; 60 | builder.price = copy.price; 61 | builder.purchaseDate = copy.purchaseDate; 62 | builder.zipCode = copy.zipCode; 63 | return builder; 64 | } 65 | 66 | 67 | public String getFirstName() { 68 | return firstName; 69 | } 70 | 71 | public String getLastName() { 72 | return lastName; 73 | } 74 | 75 | public String getCreditCardNumber() { 76 | return creditCardNumber; 77 | } 78 | 79 | public String getItemPurchased() { 80 | return itemPurchased; 81 | } 82 | 83 | public int getQuantity() { 84 | return quantity; 85 | } 86 | 87 | public double getPrice() { 88 | return price; 89 | } 90 | 91 | public Date getPurchaseDate() { 92 | return purchaseDate; 93 | } 94 | 95 | public String getZipCode() { 96 | return zipCode; 97 | } 98 | 99 | @Override 100 | public String toString() { 101 | return "Purchase{" + 102 | "firstName='" + firstName + '\'' + 103 | ", lastName='" + lastName + '\'' + 104 | ", creditCardNumber='" + creditCardNumber + '\'' + 105 | ", itemPurchased='" + itemPurchased + '\'' + 106 | ", quantity=" + quantity + 107 | ", price=" + price + 108 | ", purchaseDate=" + purchaseDate + 109 | ", zipCode='" + zipCode + '\'' + 110 | '}'; 111 | } 112 | 113 | public static final class Builder { 114 | private String firstName; 115 | private String lastName; 116 | private String creditCardNumber; 117 | private String itemPurchased; 118 | private int quanity; 119 | private double price; 120 | private Date purchaseDate; 121 | private String zipCode; 122 | 123 | private static final String CC_NUMBER_REPLACEMENT="xxxx-xxxx-xxxx-"; 124 | 125 | private Builder() { 126 | } 127 | 128 | public Builder firstName(String val) { 129 | firstName = val; 130 | return this; 131 | } 132 | 133 | public Builder lastName(String val) { 134 | lastName = val; 135 | return this; 136 | } 137 | 138 | 139 | public Builder maskCreditCard(){ 140 | Objects.requireNonNull(this.creditCardNumber, "Credit Card can't be null"); 141 | String last4Digits = this.creditCardNumber.split("-")[3]; 142 | this.creditCardNumber = CC_NUMBER_REPLACEMENT+last4Digits; 143 | return this; 144 | } 145 | 146 | public Builder creditCardNumber(String val) { 147 | creditCardNumber = val; 148 | return this; 149 | } 150 | 151 | public Builder itemPurchased(String val) { 152 | itemPurchased = val; 153 | return this; 154 | } 155 | 156 | public Builder quanity(int val) { 157 | quanity = val; 158 | return this; 159 | } 160 | 161 | public Builder price(double val) { 162 | price = val; 163 | return this; 164 | } 165 | 166 | public Builder purchaseDate(Date val) { 167 | purchaseDate = val; 168 | return this; 169 | } 170 | 171 | public Builder zipCode(String val) { 172 | zipCode = val; 173 | return this; 174 | } 175 | 176 | public Purchase build() { 177 | return new Purchase(this); 178 | } 179 | } 180 | } 181 | -------------------------------------------------------------------------------- /gradlew: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | ############################################################################## 4 | ## 5 | ## Gradle start up script for UN*X 6 | ## 7 | ############################################################################## 8 | 9 | # Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. 10 | DEFAULT_JVM_OPTS="" 11 | 12 | APP_NAME="Gradle" 13 | APP_BASE_NAME=`basename "$0"` 14 | 15 | # Use the maximum available, or set MAX_FD != -1 to use that value. 16 | MAX_FD="maximum" 17 | 18 | warn ( ) { 19 | echo "$*" 20 | } 21 | 22 | die ( ) { 23 | echo 24 | echo "$*" 25 | echo 26 | exit 1 27 | } 28 | 29 | # OS specific support (must be 'true' or 'false'). 30 | cygwin=false 31 | msys=false 32 | darwin=false 33 | case "`uname`" in 34 | CYGWIN* ) 35 | cygwin=true 36 | ;; 37 | Darwin* ) 38 | darwin=true 39 | ;; 40 | MINGW* ) 41 | msys=true 42 | ;; 43 | esac 44 | 45 | # Attempt to set APP_HOME 46 | # Resolve links: $0 may be a link 47 | PRG="$0" 48 | # Need this for relative symlinks. 49 | while [ -h "$PRG" ] ; do 50 | ls=`ls -ld "$PRG"` 51 | link=`expr "$ls" : '.*-> \(.*\)$'` 52 | if expr "$link" : '/.*' > /dev/null; then 53 | PRG="$link" 54 | else 55 | PRG=`dirname "$PRG"`"/$link" 56 | fi 57 | done 58 | SAVED="`pwd`" 59 | cd "`dirname \"$PRG\"`/" >/dev/null 60 | APP_HOME="`pwd -P`" 61 | cd "$SAVED" >/dev/null 62 | 63 | CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar 64 | 65 | # Determine the Java command to use to start the JVM. 66 | if [ -n "$JAVA_HOME" ] ; then 67 | if [ -x "$JAVA_HOME/jre/sh/java" ] ; then 68 | # IBM's JDK on AIX uses strange locations for the executables 69 | JAVACMD="$JAVA_HOME/jre/sh/java" 70 | else 71 | JAVACMD="$JAVA_HOME/bin/java" 72 | fi 73 | if [ ! -x "$JAVACMD" ] ; then 74 | die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME 75 | 76 | Please set the JAVA_HOME variable in your environment to match the 77 | location of your Java installation." 78 | fi 79 | else 80 | JAVACMD="java" 81 | which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. 82 | 83 | Please set the JAVA_HOME variable in your environment to match the 84 | location of your Java installation." 85 | fi 86 | 87 | # Increase the maximum file descriptors if we can. 88 | if [ "$cygwin" = "false" -a "$darwin" = "false" ] ; then 89 | MAX_FD_LIMIT=`ulimit -H -n` 90 | if [ $? -eq 0 ] ; then 91 | if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then 92 | MAX_FD="$MAX_FD_LIMIT" 93 | fi 94 | ulimit -n $MAX_FD 95 | if [ $? -ne 0 ] ; then 96 | warn "Could not set maximum file descriptor limit: $MAX_FD" 97 | fi 98 | else 99 | warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT" 100 | fi 101 | fi 102 | 103 | # For Darwin, add options to specify how the application appears in the dock 104 | if $darwin; then 105 | GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\"" 106 | fi 107 | 108 | # For Cygwin, switch paths to Windows format before running java 109 | if $cygwin ; then 110 | APP_HOME=`cygpath --path --mixed "$APP_HOME"` 111 | CLASSPATH=`cygpath --path --mixed "$CLASSPATH"` 112 | JAVACMD=`cygpath --unix "$JAVACMD"` 113 | 114 | # We build the pattern for arguments to be converted via cygpath 115 | ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null` 116 | SEP="" 117 | for dir in $ROOTDIRSRAW ; do 118 | ROOTDIRS="$ROOTDIRS$SEP$dir" 119 | SEP="|" 120 | done 121 | OURCYGPATTERN="(^($ROOTDIRS))" 122 | # Add a user-defined pattern to the cygpath arguments 123 | if [ "$GRADLE_CYGPATTERN" != "" ] ; then 124 | OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)" 125 | fi 126 | # Now convert the arguments - kludge to limit ourselves to /bin/sh 127 | i=0 128 | for arg in "$@" ; do 129 | CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -` 130 | CHECK2=`echo "$arg"|egrep -c "^-"` ### Determine if an option 131 | 132 | if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then ### Added a condition 133 | eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"` 134 | else 135 | eval `echo args$i`="\"$arg\"" 136 | fi 137 | i=$((i+1)) 138 | done 139 | case $i in 140 | (0) set -- ;; 141 | (1) set -- "$args0" ;; 142 | (2) set -- "$args0" "$args1" ;; 143 | (3) set -- "$args0" "$args1" "$args2" ;; 144 | (4) set -- "$args0" "$args1" "$args2" "$args3" ;; 145 | (5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;; 146 | (6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;; 147 | (7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;; 148 | (8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;; 149 | (9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;; 150 | esac 151 | fi 152 | 153 | # Split up the JVM_OPTS And GRADLE_OPTS values into an array, following the shell quoting and substitution rules 154 | function splitJvmOpts() { 155 | JVM_OPTS=("$@") 156 | } 157 | eval splitJvmOpts $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS 158 | JVM_OPTS[${#JVM_OPTS[*]}]="-Dorg.gradle.appname=$APP_BASE_NAME" 159 | 160 | exec "$JAVACMD" "${JVM_OPTS[@]}" -classpath "$CLASSPATH" org.gradle.wrapper.GradleWrapperMain "$@" 161 | -------------------------------------------------------------------------------- /src/main/java/bbejeck/streams/twitter/TwitterDataSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Bill Bejeck 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package bbejeck.streams.twitter; 18 | 19 | import com.twitter.hbc.ClientBuilder; 20 | import com.twitter.hbc.core.Client; 21 | import com.twitter.hbc.core.Constants; 22 | import com.twitter.hbc.core.Hosts; 23 | import com.twitter.hbc.core.HttpHosts; 24 | import com.twitter.hbc.core.endpoint.StatusesFilterEndpoint; 25 | import com.twitter.hbc.core.processor.StringDelimitedProcessor; 26 | import com.twitter.hbc.httpclient.auth.Authentication; 27 | import com.twitter.hbc.httpclient.auth.OAuth1; 28 | import org.apache.kafka.clients.producer.KafkaProducer; 29 | import org.apache.kafka.clients.producer.Producer; 30 | import org.apache.kafka.clients.producer.ProducerRecord; 31 | 32 | import java.io.File; 33 | import java.io.FileInputStream; 34 | import java.io.IOException; 35 | import java.util.Arrays; 36 | import java.util.List; 37 | import java.util.Properties; 38 | import java.util.concurrent.BlockingQueue; 39 | import java.util.concurrent.LinkedBlockingQueue; 40 | 41 | 42 | 43 | /** 44 | * User: Bill Bejeck 45 | * Date: 4/20/16 46 | * Time: 10:52 PM 47 | */ 48 | public class TwitterDataSource { 49 | 50 | private Client streamSource = null; 51 | private Producer producer = null; 52 | 53 | public static void main(String[] args) throws Exception { 54 | TwitterDataSource twitterDataSource = new TwitterDataSource(); 55 | twitterDataSource.run(); 56 | } 57 | 58 | public void run() { 59 | 60 | File propsFile = new File("src/main/resources/twitter-app.properties"); 61 | if (!propsFile.exists()) { 62 | System.out.println("Need to have twitter-app.properties, is it still named .template"); 63 | System.exit(1); 64 | } 65 | 66 | Properties props = new Properties(); 67 | try { 68 | props.load(new FileInputStream(propsFile)); 69 | } catch (IOException e) { 70 | throw new RuntimeException(e); 71 | } 72 | 73 | 74 | 75 | try { 76 | BlockingQueue twitterStreamQueue = new LinkedBlockingQueue<>(); 77 | streamSource = getTwitterClient(props, twitterStreamQueue); 78 | producer = getKafkaProducer(); 79 | 80 | 81 | int maxMessages = 500000; 82 | int counter = 0; 83 | 84 | streamSource.connect(); 85 | 86 | while (counter++ < maxMessages) { 87 | String twitterMessage = null; 88 | try { 89 | twitterMessage = twitterStreamQueue.take(); 90 | } catch (InterruptedException e) { 91 | Thread.currentThread().interrupt(); 92 | } 93 | ProducerRecord message = new ProducerRecord<>("twitterData", twitterMessage); 94 | producer.send(message); 95 | } 96 | 97 | } finally { 98 | stop(); 99 | } 100 | } 101 | 102 | public void stop() { 103 | if (streamSource != null) { 104 | streamSource.stop(); 105 | System.out.println("twitter streams stopped"); 106 | } 107 | if (producer != null) { 108 | producer.close(); 109 | System.out.println("kafka producer closed"); 110 | } 111 | } 112 | 113 | private Client getTwitterClient(Properties props, BlockingQueue messageQueue) { 114 | 115 | String clientName = props.getProperty("clientName"); 116 | String consumerKey = props.getProperty("consumerKey"); 117 | String consumerSecret = props.getProperty("consumerSecret"); 118 | String token = props.getProperty("token"); 119 | String tokenSecret = props.getProperty("tokenSecret"); 120 | List searchTerms = Arrays.asList(props.getProperty("searchTerms").split(",")); 121 | 122 | Authentication authentication = new OAuth1(consumerKey,consumerSecret,token,tokenSecret); 123 | Hosts hosebirdHosts = new HttpHosts(Constants.STREAM_HOST); 124 | StatusesFilterEndpoint hosebirdEndpoint = new StatusesFilterEndpoint(); 125 | 126 | hosebirdEndpoint.trackTerms(searchTerms); 127 | 128 | ClientBuilder clientBuilder = new ClientBuilder(); 129 | clientBuilder.name(clientName) 130 | .hosts(hosebirdHosts) 131 | .authentication(authentication) 132 | .endpoint(hosebirdEndpoint) 133 | .processor(new StringDelimitedProcessor(messageQueue)); 134 | 135 | return clientBuilder.build(); 136 | 137 | } 138 | 139 | private Producer getKafkaProducer() { 140 | Properties props = new Properties(); 141 | props.put("bootstrap.servers", "localhost:9092"); 142 | props.put("acks", "all"); 143 | props.put("retries", 0); 144 | props.put("batch.size", 16384); 145 | props.put("linger.ms", 1); 146 | props.put("buffer.memory", 33554432); 147 | props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); 148 | props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); 149 | 150 | return new KafkaProducer<>(props); 151 | } 152 | } 153 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "{}" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright {yyyy} {name of copyright owner} 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | -------------------------------------------------------------------------------- /src/main/resources/kafkaStreamsTwitterTrainingData_clean.csv: -------------------------------------------------------------------------------- 1 | spanish#Hacia la mitad de la calle de la Universidad, entre los números 51 y 57, 2 | spanish#se ven cuatro hoteles que pueden citarse entre los más lindos de París. 3 | spanish#El primero pertenece al señor Pozzo di Borgo, el segundo al condeMailly, el tercero al duque de Choiseul y el último, que hace esquina ala calle Bellechasse, al barón de Sanglié. 4 | spanish#El aspecto de este edificio es noble. La puerta cochera da entrada a unpatio de honor cuidadosamente enarenado y tapizado de parrascentenarias. El pabellón del portero está a la izquierda, envuelto entreel follaje espeso de la hiedra, donde los gorriones y los huéspedes dela garita parlotean al unísono. En el fondo del patio, a la derecha, unaamplia escalinata resguardada por una marquesina, conduce al vestíbulo ya la gran escalera. 5 | spanish#La planta baja y el primer piso están ocupados por el barón únicamente, 6 | spanish#que disfruta sin compartirlo con nadie un vasto jardín, limitado porotros jardines, y poblado de urracas, mirlos y ardillas que van y vienende ése a los otros en completa libertad, como si se tratara dehabitantes de un bosque y no de ciudadanos de París. 7 | spanish#Las armas de los Sanglié, pintadas en negro, se descubren en todas lasparedes del vestíbulo. Son un jabalí de oro en un campo de gules. Elescudo tiene por soporte dos lebreles, y está rematado con el penacho debarón con esta leyenda: _Sang lié au Roy_[A]. 8 | spanish#Como media docena de lebreles vivos, agrupados según su capricho, seaburren al pie de la escalera, mordisquean las verónicas floridas en losvasos del Japón o se tienden sobre la alfombra alargando la cabezaserpentina. Los lacayos, sentados en banquetas de Beauvais, cruzansolemnemente los brazos, como conviene a los criados de buena casa. 9 | spanish#El día 1.º de enero de 1853, hacia las nueve de la mañana, toda laservidumbre del hotel celebraba en el vestíbulo un congreso tumultuoso. 10 | spanish#El administrador del barón, el señor Anatolio, acababa de distribuirlesel aguinaldo. El mayordomo había recibido quinientos francos, el ayudade cámara doscientos cincuenta. El menos favorecido de todos, elmarmitón, contemplaba con una ternura inefable dos hermosos luises deoro completamente nuevos. Habría celosos en la asamblea, perodescontentos ni uno solo, y cada uno a su manera decía que da gustoservir a un amo rico y generoso. 11 | spanish#Los tales individuos formaban un grupo bastante pintoresco alrededor deuna de las bocas del calorífero. Los más madrugadores llevaban ya lagran librea; los otros vestían aún el chaleco con mangas que constituyeel uniforme de media gala de los criados. 12 | spanish#El ayuda de cámara iba vestido de negro completamente, con zapatillas deorillo; el jardinero parecía un aldeano endomingado; el cochero llevabachaqueta de tricot y sombrero galoneado; el portero un tahalí de oro yzuecos. Aquí y acullá se distinguía a lo largo de las paredes, unafusta, una almohaza, un encerador, escobas, plumeros y algo más cuyonombre ignoro. 13 | spanish#El señor dormía hasta mediodía, como quien ha pasado la noche en elclub, y por lo tanto tenían tiempo para empezar sus faenas. Por lopronto se entretenían en darle empleo al dinero y las ilusiones lesocupaban bastante. Los hombres todos son algo parientes de aquellalechera de la fábula. 14 | spanish#Con esto, y lo que ya tengo ahorradodecía el mayordomo, puedoredondear mi renta vitalicia. A Dios gracias no falta el pan, y los díasde la vejez los tendré asegurados. 15 | spanish#Como es usted solteroreplico el ayuda de cámara, no tiene quepensar en nadie. Pero yo tengo familia. Por eso pienso entregarle eldinero a ese buen señor que va a la Bolsa, y algo me producirá. 16 | spanish#Es una buena idea, señor Fernandodijo el marmitón. Cuando vayausted, llévele mis cuarenta francos. 17 | spanish#El ayuda de cámara se creyó obligado también a intervenir y exclamó entono de protección: 18 | spanish#¡Vaya con el joven! ¿Qué crees tú que se puede hacer con cuarentafrancos en la Bolsa? 19 | spanish#Buenorespondió el joven ahogando un suspiro, los llevaré a la Cajade ahorros. 20 | spanish#El cochero soltó una ruidosa carcajada y se dio unos puñetazos sobre elestómago gritando: 21 | spanish#Esta es mi caja de ahorros. Aquí es donde he colocado siempre misfondos, y a fe que no me ha ido mal. ¿Verdad, padre Altorf? 22 | spanish#El padre Altorf, suizo[B] de profesión, alsaciano de nacimiento, deelevada estatura, vigoroso, huesudo, de desarrollado vientre, ancho dehombros, de cabeza enorme y rubicundo como un hipopótamo, sonrió con elrabillo del ojo y produjo con la lengua un pequeño chasquido que eratodo un poema. 23 | spanish#El jardinero, delicada flor de la Normandía, hizo sonar el dinero en sumano y respondió al honorable preopinante: 24 | spanish#¡Vamos, no diga usted tonterías! lo que se ha bebido ya no se vuelve atener. Lo mejor que hay es esconder el dinero en una pared vieja o en unárbol hueco. ¡Los que así lo hagan no darán de comer al notario! 25 | spanish#La asamblea en pleno protestó de la ingenuidad de aquel buen hombre queenterraba en flor sus escudos, sin hacerlos producir. Quince o diez yseis exclamaciones se elevaron al mismo tiempo. Cada uno expuso suopinión, descubrió su secreto, cabalgó en su Clavileño. Cada uno hizosaltar las monedas en su bolsillo y acarició ardientemente lasesperanzas ciertas, la dicha contante y sonante que habían embolsado. Eloro mezclaba su aguda vocecita con aquel concierto de pasiones vulgares; 26 | spanish#y el choque de las piezas de veinte francos, más embriagador que losvapores del vino o el olor de la pólvora, emborrachaba a aquellos pobrescerebros y aceleraba los latidos de sus groseros corazones. 27 | spanish#En lo más fuerte del tumulto, se abrió una pequeña puerta que daba a laescalera, entre el piso bajo y el primero. Una mujer, con un harapientotraje negro, descendió vivamente los peldaños, atravesó el vestíbulo, 28 | spanish#abrió la puerta de vidrieras y desapareció en el patio. 29 | spanish#Todo esto pasó en un minuto y, no obstante, la sombría aparición sellevó el buen humor de todas aquellas gentes, que se levantaron a supaso con el más profundo respeto. Los gritos se detuvieron en susgargantas y el oro ya no volvió a sonar en sus bolsillos. La pobre mujerhabía dejado detrás de ella como una estela de silencio y de estupor. 30 | spanish#El primero que se repuso fue el ayuda de cámara, que era lo que se llamaun espíritu fuerte. 31 | spanish#¡Voto a...!exclamó. He creído ver pasar a la miseria en persona. 32 | spanish#e ha estropeado el año. Ya veréis cómo no vuelve a salirme nada bienhasta el día de San Silvestre. ¡Brrr! tengo frío en la espalda. 33 | spanish#¡Pobre mujer!dijo el mayordomo. Ha tenido cientos y miles y ya laveis ahora... ¿Quién creería que es una duquesa? 34 | spanish#Es que el vagabundo de su marido se lo ha comido todo. 35 | spanish#¡Un jugador! 36 | spanish#¡Un hombre que no piensa más que en comer! 37 | spanish#Un andariego que trota de la mañana a la noche, con sus piernas derocín. 38 | spanish#No es él el que me interesa: tiene lo que se merece. 39 | spanish#¿Se sabe algo de la señorita Germana? 40 | spanish#Su negra me ha dicho que cada día está peor. A cada golpe de tos llenaun pañuelo. 41 | spanish#¡Y sin una alfombra en su habitación! Esa niña no se curaría más queen un país templado, en Italia, por ejemplo. 42 | spanish#Será un ángel para Dios. 43 | spanish#Los que quedan son más dignos de compasión. 44 | spanish#¡No sé cómo se las arreglará la duquesa para salir de este atolladero. 45 | spanish#¡A todos debe! Ultimamente el panadero se ha negado a fiarles más. 46 | spanish#¿Cuánto deben de alquiler? 47 | spanish#Ochocientos francos; pero lo que me extraña es que siquiera el señorhaya visto el color de su dinero. 48 | spanish#Si yo fuese él, preferiría tener desalquilado el piso antes quepermitir que viviesen en él personas que deshonran la casa. 49 | spanish#¡No seas bestia! ¿Para qué arrastrar por el arroyo al duque de La Tourde Embleuse y a su familia? Esas miserias, para que lo sepas, son comolas llagas del barrio; todos nosotros tenemos interés en ocultarlas. 50 | spanish#¡Toma!dijo el marmitón, creo que tengo razón para burlarme. ¿Porqué no trabajan? Los duques son hombres como los demás. 51 | spanish#¡Muchacho!exclamó gravemente el mayordomo, estás diciendo cosasincoherentes. La prueba de que no son hombres como los demás, es que yo, 52 | spanish#tu superior, no sería ni barón durante una hora de mi vida. Además, laduquesa es una mujer sublime y hace cosas de las que ni tú ni yoseríamos capaces. ¿Tomarías tú caldo durante todo un año y en todas lascomidas? 53 | spanish#¡Caramba! ¡No me parece eso muy divertido! 54 | spanish#¡Pues bien! la duquesa pone el puchero a la lumbre cada dos días, 55 | spanish#porque a su marido no le gusta la sopa de vigilia. El señor se come sutapioca de caldo graso y un bistec y un par de chuletas, y la pobre ysanta mujer se conforma con los desperdicios. Es hermoso, ¿verdad? 56 | german#Bär, der große, alte, struppige Hofhund auf Hoël, saß auf der Türtreppeund schaute ernsthaft über den Hof. Es war ein kalter, klarerSpätwintertag, und der Schnee glitzerte im Sonnenschein. Am liebstenwäre Bär aber doch hineingegangen; denn es ließ sich nicht leugnen: wieer da saß, fror ihn grimmig an den Pfoten, und er hob abwechselnd balddie eine, bald die andre eine Weile von den Steinfliesen empor, um nichtdas Kribbeln in die Klauen zu bekommen. 57 | german#Aber er durfte seinen Posten nicht verlassen. Die Schweine und dieZiegen waren heute im Freien. Noch führten sie sich zwar alle ganzanständig auf; die Schweine gingen dort in der Sonne und rieben sich ander Ecke des Kuhstalls, und weiter weg knabberten die Ziegen eifrig ander Baumrinde, die beim Schweinestall auf einen großen Haufen für siezusammengekehrt war, und taten so, als hätten sie an nichts andres zudenken. Aber er wußte von früher her: kaum war er hineingegangen, dalagerten sie sich sogleich mitten in den Haustüren und verübten all denUnfug, den sie sich nur ausdenken konnten die große neue ZiegeKrummhorn, die erst im letzten Herbst auf den Hof gekommen war, und dieer noch nicht ordentlich abgerichtet, hatte bereits einen Schwuppdichbis an die Hausecke hin gemacht und ihn dabei so gleichgültig undüberlegen angesehen. 58 | german#Die war wirklich eine unerträgliche Person, aber sie sollte sich bloß 59 | german#unterstehen ! 60 | german#Ein Weilchen wenigstens mußte er noch sitzen bleiben die Wege durfteer ja auch nicht ganz aus dem Auge verlieren, es hätte doch jemandkommen können. 61 | german#Zufällig drehte er den Kopf nach dem schmalen Pfad hin, der die Haldeschräg vom Oberdorf herabkam. 62 | german#Alle Wetter! Was war denn das? 63 | german#Dort kam Etwas etwas Rundes, Putziges, Winziges ärgerlich, daß dieAugen nicht mehr recht mitwollten! ja, ja, er mußte auf alle FälleMeldung machen. 64 | german#Er fing an zu bellen, ein kurzes, tiefes Gekläff, das weithinausschallte. Die Ziegen sprangen ängstlich in einen Klumpen zusammenund spitzten die Ohren, die Schweine hörten jählings auf, sich zu juckenund zu kratzen, und lauschten ja, da konnte man sehen, daß sie vorihm Furcht hatten. 65 | german#Dann blieb er wieder ruhig sitzen und sah den Weg hinauf. Nein aber, ober jemals etwas Ähnliches gesehen hatte vielleicht war es nichteinmal etwas, das er zu melden brauchte; aber immerhin mußte er sichwohl auf den Weg machen und sich die Sache etwas näher ansehen. 66 | german#Er krümmte den buschigen Schwanz in einen großen Bogen; man solltesehen, daß er bester Laune war, und trippelte zum Hofe hinaus. 67 | german#Es mußte aber doch wohl ein Mensch sein. Es fing an, so leibhaftig derFinn-Kathrine zu gleichen, die dort im Winter zu gehen pflegte, aber diekonnte es doch nicht sein; denn dazu war das Wesen dort allzu winzig. 68 | german#Aber ein weiter, langer Weiberrock war es jedenfalls, und unter dem Rockkamen die Spitzen von einem Paar großer Schuhe hervor, über die graue, 69 | german#abgeschnittne Strumpffüße gezogen waren. Über den Rock war ein großerBausch Gestricktes gewurstelt, aus dem zwei Stummel mit roten, 70 | german#gestrickten Fausthandschuhen hervorguckten. Oben drauf saß ein etwaskleinerer Bausch Gestricktes das war wohl der Kopf. Hinten auf demRücken hing ein großes Bündel in einem dunkelfarbigen Einschlagetuch undvorn ein kleiner, niedlicher, rotgemalter Holzeimer. 71 | german#Bär mußte unwillkürlich stehen bleiben und sehen. Das rätselhafte Wesenwar nun ebenfalls seiner gewahr geworden und wie unschlüssig stehengeblieben. Da ging er auf die äußerste Wegkante hinüber, blieb dortstehen und versuchte, so gleichgültig wie möglich auszusehen, um dasWesen nicht zu erschrecken. Dies ging dann vorsichtig, wie auf Stelzen, 72 | german#langsam wieder vorwärts, indem es sich dicht an der andern Seite desWeges hindrückte und drehte sich allmählich, je näher es herankam, sodaß es ganz der Quere ging, als es endlich gerade vor Bär angelangt war. 73 | german#Da gelang es aber Bär, einen kurzen Blick durch eine kleine Öffnung indem obersten Strickbausch zu werfen, und was sah er! Erst ein kleines, 74 | german#rotes, aufwärtsstrebendes Stumpfnäschen, dann einen roten Mund, derunsicher zuckte, als wollte er zu weinen anfangen, und ein Paar großeblaue Augen, die ihn erschrocken anstarrten. 75 | german#Bah! Das war ja bloß ein kleines Mädel, das wegen der Kälte tüchtigeingemummelt war. Er kannte sie zwar nicht, aber wart mal dasEimerchen kam ihm so bekannt vor. Jedenfalls war es keine Art, sich hierbarsch zu stellen und so ein kleines Ding zu erschrecken. 76 | german#Unwillkürlich wedelte er mit dem Schwanze während er hinüberging, um denEimer zu beschnüffeln. 77 | german#Aber das kleine Mädchen verstand ihn nicht sofort, erschrocken trat esvielmehr ein paar Schritte zurück und purzelte rücklings neben dem Wegehin. Da sprang Bär rasch zur Seite und lief ein Stückchen voraus, sahsich wieder um und blinzelte freundlich mit den Augen und wedeltekräftig mit dem Schwanze. Jetzt begriff sie, stand auf, lächelte undtrippelte hinter ihm drein. Bär humpelte voran, sich immer wiederumsehend; nun erkannte er, daß sie sicher irgend einen Auftrag auf Hoëlauszurichten hatte, und da war es seine einfache Pflicht undSchuldigkeit, ihr zurecht zu helfen. 78 | german#Das kleine Mädchen war Sidsel Langröckchen von Schloß Guckaus oben aufder Höhe, die dergestalt auf Hoël ihren Einzug hielt. 79 | german#Schloß Guckaus lag auf einem öden, unfruchtbaren Berghang, weit vorn, 80 | german#gerade unter dem Großhammer, zu alleroberst im Oberdorf, und der Name 81 | german#es hieß eigentlich Neu-Wüstenland war ein Spitzname, den ihm einSpottvogel gegeben, weil man von da oben einen weiten Ausguck hatte, undweil es allem andern eher als einem Schloß glich. Das Krongut, das zumSchloß gehörte, bestand bloß aus etwas Heideland, wo Heidel- undPreiselbeerkraut üppig gedieh, unterbrochen hier und da von einemkleinen Fleckchen Ackerboden oder einem Stückchen Wiese. 82 | german#Die Stallgebäude bestanden aus einem untermauerten Kuhstall mit zweiStänden, halb in den Hügel eingegraben, und einem kleinen Schweinekofenim gleichen Stil. Und das Schloß selbst war ein winzig kleines, mitRasen gedecktes Häuschen, das ganz vorn am Abhang mitten in der Einödelag. Es hatte bloß ein niedriges Fensterchen mit ganz kleinen Scheiben, 83 | german#das ins Tal hinabschaute. 84 | german#Fast überall aber, wo man im Umkreis sein mochte wenn man in derRichtung hinsah und den Blick hoch genug hinaufwandte, überall sah manstets dieses Schloß und dies Guckfensterchen, das wie ein kleines Augeüber das Tal hinausblickte. 85 | german#Wenn nun die Herrlichkeit, von der sie herkam, nicht größer war, so kannman sich wohl leicht denken, daß Sidsel Langröckchen just keineverkleidete Prinzessin war, sondern schlecht und recht ein kleines armesBettelkind. Und zum ersten Mal auf den Hof von Hoël zu kommen, war fürsie dasselbe, als wenn sie wirklich zu Hofe gekommen wäre, obschon siein einem gar wichtigen, eigentlich nur für Erwachsene passenden Auftraggeschickt war; sie kam nämlich an Stelle ihrer Mutter als Spinnfrau. 86 | german#Sidsels Mutter, Rönnaug, hatte nun schon vier Jahre lang oben auf Schloß 87 | german#Guckaus allein für den Unterhalt der Familie sorgen müssen. Früher wares ihnen gut gegangen; da war aber der Mann gestorben, und nun saß sieallein da mit dem Schloß, einer Kuh und zwei Kindern, Jakob, der damalsungefähr sechs Jahre zählte, und Sidsel, die zwei Jahre jünger war. Eshielt oft schwer genug, aber sie hatten doch immerhin ein Dach über demKopfe, und nach Brennholz brauchten sie auch nicht weit zu laufen, derWald lag gerade vor der Tür. 88 | german#Im Sommer konnte sie den harten, steinigen Boden gerade soweitaufkratzen, daß sie auf dem jämmerlich kleinen Fleckchen AckerlandKartoffeln und etwas Korn bauen konnte, und Heidegras und frisches Laub, 89 | german#das sie sammelte, gab es gerade genug, um die Kuh Bliros jedes Jahrdurchfüttern zu können. Und wo eine Kuh ist, da gibt's auch immer was zuleben. 90 | german#Im Winter spann sie fleißig Leinwand und Wolle für die Bäuerinnen untenim Dorfe und vor allem für Kjersti Hoël, die Großbäuerin, bei der sieals Magd gedient hatte, ehe sie sich verheiratete. 91 | german#Auf diese Weise hatte sie sich durchgeschlagen. Unterdessen war derJakob so groß geworden, daß er selber für sich sorgen konnte. Im letztenFrühling war auf Nordrum Nachfrage nach einem Hirtenbuben gewesen, undda hatte er sofort zugeschlagen. Er und Sidsel hatten so oft oben in derStube vor dem kleinen Guckfensterchen auf den Knieen gelegen, 92 | german#hinausgeschaut und überlegt, wo sie wohl beide einmal dienen würden, 93 | german#wenn sie erst groß wären. Und da hatte der Knabe immer Nordrum für sichgewählt, hauptsächlich deshalb, weil er den Bauer von Nordrum immer alseinen besonders starken Mann hat rühmen hören, sie dagegen hattegemeint, besser müsse es auf Hoël sein, wo bloß Frauensleute wären. 94 | german#Im Herbst hatte dann der Nordrum gesagt, so einen Burschen wie den Jakobkönne er auch im Winter brauchen, und da war Jakob dort geblieben. 95 | german#Er war sogar schon letzte Weihnachten einen ganzen Tag wieder zu Hausegewesen, und da hatte er der Schwester ein Weihnachtsgeschenkmitgebracht von einem kleinen Mädchen auf dem Nordrumhofe, einen garfeinen Unterrock aus grauem Fries. 96 | german#Und wie lustig und spaßig er geworden war! Als sie den neuen Rock, derihr vorn wie hinten bis ganz herunter auf die Füße reichte, zumerstenmal angezogen hatte, da hatte er sie Sidsel _Langröckchen_ 97 | german#genannt. 98 | german#Nach Weihnachten aber war oben auf Schloß Guckaus die Not eingezogen. 99 | german#Die Kuh Bliros, die sonst fast das ganze Jahr hindurch Milch gab, ließ 100 | german#es sich plötzlich einfallen, mehrere Monate trocken zu stehen; siesollte erst zum Sommer hin kalben. Die letzte Woche hatte es nichteinmal mehr Milch zum Kaffee gegeben. 101 | german#Bis zum Nachbarhof Svehaugen war es auch nicht bloß ein Katzensprung, 102 | german#und dort war es zudem auch knapp mit der Milch, das wußte Rönnaug, undaußerdem hatte sie keine Zeit, sie mußte sich sputen, daß sie mit derWolle, die sie für Kjersti Hoël spann, endlich fertig wurde und sie baldabliefern konnte, dann wurde wohl auch Rat für Milch und Kaffee undandres mehr. Deshalb arbeitete sie unausgesetzt die ganze Woche hindurch Sidsel war nun so groß, daß sie beim Karden helfen konnte undtrank den Kaffee schwarz. War es nun aber, weil sie den schwarzen Kaffeenicht vertragen konnte, oder ein andrer Grund, als sie gestern abendspät fertig geworden war, fühlte sie einen saugenden Schmerz unter derBrust, und als sie heute früh aufstand und sich fertig machte, mit derWolle nach Hoël zu gehen, wurde ihr mit einem Mal so übel undschwindlig, daß sie sich wieder aufs Bett legen mußte. Sie fühlte sichganz elend. Nun war es aber Sitte, daß die Spinnfrau, was siegesponnnen, auch selber brachte, und da bekam sie nicht bloß Vergütungfür ihre Arbeit, sondern wurde auch bewirtet und erhielt neue Aufträgeund Bescheid, wie das nächste Garn gesponnen werden sollte. 103 | german#Doch diesmal war wirklich kein andrer Ausweg, sie mußte Sidsel schicken. 104 | german#Sie würde sich schon zurechtfinden, obwohl sie noch nie auf Hoël gewesenwar, und soviel würde sie wohl auch mit nach Hause bringen, daß siewenigstens wieder einmal eine ordentliche Tasse Kaffee trinken könnten, 105 | german#dann konnte sie ja an einem der nächsten Tage immer noch selber gehen. 106 | german#Wenn sie sich nur darauf verlassen könnte, daß Sidsel sich ordentlich zubenehmen verstände und sich nicht gar zu ungelenk anstellte? 107 | german#O ja, hatte Sidsel gemeint, wenn sie nur gehen dürfte, dann würde siesich schon richtig zu benehmen wissen, genau wie eine Spinnfrau; dennsie erinnerte sich sehr gut daran, wie die es machten, von damals, alssie die Mutter nach Nordrum begleiten durfte. 108 | german#Da hatte denn Sidsel ihren Fries-Unterrock angezogen sie trug ihn wieeinen richtigen Rock und nur bei festlichen Gelegenheiten warordentlich in eine Menge Tücher und wollene Schale eingewickelt worden, 109 | german#hatte das große Garnknäuel hinten auf den Rücken gehängt bekommen, denMilcheimer vorn um den Hals und außerdem viele gute Lehren undErmahnungen mit auf den Weg, und so ging es zu, daß sie heute hinter Bärhertrippelnd in den Hof von Hoël einzog. 110 | german#Als sie an den Wirtschaftsgebäuden vorüberkam, mußte sie wirklich stehenbleiben und sich umsehen. Ja, hier war freilich alles viel großartigerals daheim! Ein Scheunentor so breit, daß das ganze Guckaus-Schloß hättehindurchfahren können, und eine einzige Fensterscheibe war mindestensebenso groß, wie das ganze Fenster oben im Schloß! Und solch eine Ziege! 111 | german# Sie bekam gerade Krummhorn zu Gesicht, die der Haustür bereitsbedenklich nahe war und kaum Miene machte auszureißen, als Bär herankam. 112 | german#Nicht einmal vor dem Hunde war die bange! Sie war freilich auch so groß 113 | german#wie ein ordentliches Kalb! Wenn die Kühe nach demselben Maßstabe waren, 114 | german#da mußten sie ja vom Erdboden aus das Gras auf dem Dache von Schloß 115 | german#Guckaus fressen können! Sie mußte unwillkürlich nach der Tür desKuhstalls hinschielen. Nein, die war doch nicht größer als Stalltürengewöhnlich sind da waren die Kühe doch wohl wie andre auch. 116 | german#Bär hatte unterdessen die Zeit benutzt, um Krummhorn zurecht zu weisen, 117 | german#nun kam er zurück, wedelte lustig mit dem Schwanze und wandte sich nachder Haustür, als ob er die Kleine hineingeleiten wollte. Ja, der Hundhatte recht, sie mußte sich wirklich beeilen, um ihren Auftragauszurichten und durfte nicht länger hier stehen und gaffen. 118 | german#Sie folgte Bär nach, ging in die Hausflur hinein, hob die Türklinke, 119 | german#drehte sich ganz um sich selbst herum, als sie die Tür wieder zumachte, 120 | french#Vers le milieu de la rue de l'Université, entre le numéro 51 et le 57,on voit quatre hôtels qui peuvent compter parmi les plus beaux deParis. Le premier appartient à M. Pozzo di Borgo; le second, au comtede Mailly; le troisième, au duc de Choiseul; le dernier au baron deSanglié. C'est celui qui fait l'angle de la rue Bellechasse. 121 | french#L'hôtel de Sanglié est une habitation de noble apparence. La portecochère s'ouvre sur une cour d'honneur soigneusement sablée et tapisséede treilles centenaires. La loge du suisse est à gauche, cachée sous unlierre épais où les moineaux et les portiers babillent à l'unisson. Aufond de la cour à droite, un large perron, abrité sous une marquise,conduit au vestibule et au grand escalier. Le rez-de-chaussée et lepremier sont occupés par le baron tout seul; il jouit sans partage d'unvaste jardin borné par d'autres jardins, peuplé de fauvettes, de merleset d'écureuils qui vont de l'un chez l'autre en pleine liberté, commes'ils étaient habitants d'un bois, et non citoyens de Paris. 122 | french#Les armes des Sanglié, peintes à la cire, se répètent sur tous les mursdu vestibule. C'est un sanglier d'or sur champ de gueules. L'écusson estsupporté par deux lévriers et surmonté d'un tortil de baron avec cettelégende: SANG LIÉ AU ROY. Une demi-douzaine de lévriers vivants, groupéssuivant leur fantaisie, s'agacent au pied de l'escalier, mordillent lesvéroniques en fleur dans les vases du Japon, ou s'aplatissent sur letapis en allongeant leur tête serpentine. Les valets de pied, assis surdes banquettes de Beauvais, se croisent solennellement les bras, commeil convient à des gens de bonne maison. 123 | french#Le 1er janvier 1853, vers les neuf heures du matin, tous les domestiquesde l'hôtel tenaient sous le vestibule un congrès tumultueux. L'intendantdu baron, M. Anatole, venait de leur distribuer leurs étrennes. Lemaître d'hôtel avait reçu cinq cents francs, le valet de chambre deuxcent cinquante. Le moins favorisé de tous, le marmiton, contemplait avecune tendresse inexprimable deux beaux louis d'or tout neufs. Il y avaitdes jaloux dans l'assemblée, mais pas un mécontent, et chacun disait enson langage que c'est plaisir de servir un maître riche et généreux. 124 | french#Ces messieurs formaient un groupe assez pittoresque autour d'une desbouches du calorifère. Les plus matineux avaient déjà la grande livrée; 125 | french#les autres portaient encore le gilet à manches, qui est la petite tenuedes domestiques. Le valet de chambre était tout de noir habillé, avecdes chaussons de lisière; le jardinier ressemblait à un villageoisendimanché; le cocher était en veste de tricot et en chapeau galonné; lesuisse, en baudrier d'or et en sabots. On apercevait ça et là, le longdes murs, un fouet, une étrille, un bâton à cirer, une tête de loup, etdes plumeaux dont je ne sais pas le nombre. 126 | french#Le maître dormait jusqu'à midi, en homme qui a passé la nuit au club: 127 | french#on avait bien le temps de se mettre à l'ouvrage. Chacun faisait d'avanceemploi de son argent, et les châteaux en Espagne allaient bon train. 128 | french#Tous les hommes, petits et grands, sont de la famille de Perrette quiportait un pot au lait. 129 | french#«Avec ça et ce que j'ai de côté, disait le maître d'hôtel, j'arrondiraima rente viagère. On a du pain sur la planche, Dieu merci! et l'on ne selaissera manquer de rien sur ses vieux jours. 130 | french#Parbleu! reprit le valet de chambre, vous êtes garçon; vous n'avezque vous à penser. Mais, moi, j'ai de la famille. Aussi, je donneraimon argent à ce petit jeune homme qui va à la Bourse. Il me tripoteraquelque chose. 131 | french#C'est une idée, ça, monsieur Ferdinand, repartit le marmiton. 132 | french#Portez-lui donc mes quarante francs, quand vous irez.» 133 | french#Le valet de chambre répondit d'un ton protecteur: «Est-il jeune! 134 | french#Qu'est-ce qu'on peut faire à la Bourse avec quarante francs? 135 | french#Allons, dit le jeune homme en étouffant un soupir, je les mettrai à lacaisse d'épargne!» 136 | french#Le cocher partit d'un gros éclat de rire. Il frappa sur son estomacen criant: «Ma caisse d'épargne, à moi, la voici. C'est là que j'aitoujours placé mes fonds, et je m'en suis bien trouvé. Pas vrai, pèreAltroff?» 137 | french#Le père Altroff, suisse de profession, Alsacien de naissance, grand,vigoureux, ossu, pansu, large des épaules, énorme de la tête, et aussirubicond qu'un jeune hippopotame, sourit du coin de l'oeil et fit avecsa langue un petit bruit qui valait un long poème. 138 | french#Le jardinier, fine fleur de Normand, fit sonner son argent dans sa main,et répondit à l'honorable préopinant: «Allais, marchais! ce qu'on a bu,on ne l'a plus. Il n'est tel placement qu'une bonne cachette dans unvieux mur ou dans un arbre creux. Argent bien enfouie, les notaires nela mangent point!» 139 | french#L'assemblée se récria sur la naïveté du bonhomme qui enterrait ses écustout vifs, au lieu de les faire travailler. Quinze ou seize exclamationss'élevèrent en même temps. Chacun dit son mot, trahit son secret,enfourcha son dada, secoua sa marotte. Chacun frappa sur sa poche etcaressa bruyamment les espérances certaines, le bonheur clair et liquidequ'il avait emboursé le matin. L'or mêlait sa petite voix aiguë à ceconcert de passions vulgaires; et le cliquetis des pièces de vingtfrancs, plus capiteux que la fumée du vin ou l'odeur de la poudre,enivrait ces pauvres cervelles et accélérait le battement de ces coeursgrossiers. 140 | french#Au plus fort du tumulte, une petite porte s'ouvrit sur l'escalier, entrele rez-de-chaussée et le premier étage. Une femme, vêtue de haillonsnoirs, descendit vivement les degrés, traversa le vestibule, ouvrit laporte vitrée et disparut dans la cour. 141 | french#Ce fut l'affaire d'une minute, et pourtant cette sombre apparitionéteignit la joie de tous ces valets en belle humeur. Ils se levèrent surson passage avec les marques d'un profond respect. Les cris s'arrêtèrentdans leur gosier, et l'or ne sonna plus dans leurs poches. La pauvrefemme avait laissé derrière elle comme une traînée de silence et destupeur. 142 | french#Le premier qui se remit fut le valet de chambre, un esprit fort. 143 | french#«Sapristi! cria-t-il, j'ai cru voir passer la misère en personne. Voilà 144 | french#mon jour de l'an gâté dès le matin. Vous verrez que rien ne me réussirajusqu'à la Saint-Sylvestre. Brrr! j'ai froid dans le dos. 145 | french#Pauvre femme! dit le maître d'hôtel. Ça a eu des mille et des cents,et puis voilà! Qui est-ce qui croirait que c'est une duchesse? 146 | french#C'est son gueux de mari qui lui a tout mangé. 147 | french#Un joueur! 148 | french#Un homme sur sa bouche! 149 | french#Un coureur qui trotte du matin au soir, avec ses vieilles jambes, à lasuite de tous les cotillons! 150 | french#C'est pas lui qui m'intéresse: il n'a que ce qu'il mérite. 151 | french#Sait-on comment va Mlle Germaine? 152 | french#Leur négresse m'a dit qu'elle était au plus bas. Elle crache le sang à 153 | french#plein mouchoir. 154 | french#Et pas de tapis dans sa chambre! Cette enfant-là ne guérirait que dansles pays chauds, à Florence ou en Italie. 155 | french#Ça fera un ange au ciel du bon Dieu. 156 | french#C'est ceux qui restent qui sont à plaindre! 157 | french#Je ne sais pas comment la duchesse sortira de là. Des comptes à 158 | french#n'en plus finir chez tous les fournisseurs! Le boulanger parle de leurrefuser crédit. 159 | french#Combien ont-ils de loyer là-haut? 160 | french#Huit cents. Mais je m'étonne si monsieur à jamais vu la couleur deleur argent. 161 | french#Si j'étais de lui, j'aimerais mieux laisser le petit appartementvacant que de garder des personnes qui font tache dans l'hôtel. 162 | french#Es-tu bête! Pour qu'on ramasse sur le pavé le duc de La Tourd'Embleuse et sa famille? Ces misères-là, vois-tu, c'est comme lesplaies du faubourg: nous avons tous intérêt à les cacher. 163 | french#Tiens! dit le marmiton, je m'en moque pas mal! Pourquoi qu'ils netravaillent pas? Les ducs sont des hommes comme les autres. 164 | french#Garçon! reprit gravement le maître d'hôtel, tu dis des chosesincohérentes. La preuve qu'ils ne sont pas des hommes comme les autres,c'est que moi, ton supérieur, je ne serai pas seulement baron pendantune heure de ma vie. D'ailleurs la duchesse est une femme sublime,et elle fait des choses dont ni toi ni moi ne serions capables. 165 | french#Mangerais-tu du bouilli pendant un an à tous tes repas? 166 | french#Dame! ça n'est pas amusant, le bouilli! 167 | french#Eh bien! la duchesse met le pot-au-feu tous les deux jours, parce queson mari n'aime pas la soupe maigre. Monsieur dîne d'un bon tapioca augras, avec un bifteck ou une paire de côtelettes, et la pauvre saintefemme avale jusqu'au dernier morceau de gîte qui se bouillit dans lamaison. Est-ce beau, cela?» 168 | french#Le marmiton fut touché dans l'âme. «Mon bon monsieur Tournoy, dit-ilau maître d'hôtel, c'est des gens bien intéressants. Est-ce qu'on nepourrait pas leur faire passer quelques douceurs, en s'entendant avecleur négresse? 169 | french#Ah bien oui! elle est aussi fière qu'eux; elle ne voudrait rien denous. Et cependant m'est avis qu'elle ne déjeune pas tous les jours.» 170 | french#Cette conversation aurait pu durer longtemps, si M. Anatole n'était venul'interrompre. Il entra juste à point pour couper la parole au chasseur,qui ouvrait la bouche pour la première fois. L'assemblée se dispersa entoute hâte; chaque orateur emporta ses instruments de travail, et il neresta dans la salle des délibérations qu'un de ces balais gigantesquesqu'on appelle tête de loup. 171 | french#Cependant Marguerite de Bisson, duchesse de La Tour d'Embleuse,cheminait à pas pressés dans la direction de la rue Jacob. Les passantsqui la frôlèrent du coude en courant donner ou recevoir des étrennes latrouvèrent semblable à ces Irlandaises désespérées qui piétinent sur lemacadam des rues de Londres à la poursuite d'un penny. Fille des ducsde Bretagne, femme d'un ancien gouverneur du Sénégal, la duchesseétait coiffée d'un chapeau de paille teinte en noir, dont les brides setordaient comme des ficelles. Une voilette d'imitation, percée en cinqou six endroits, cachait mal son visage et lui donnait une physionomieétrange. Cette belle tête, marquée de taches blanches d'inégalegrandeur, semblait défigurée par la petite vérole. Un vieux crêpe deChine, noirci par les soins du teinturier et roussi par les intempériesde l'air, laissait tomber tristement ses trois pointes, dont la frangeeffleurait la neige du trottoir. La robe qui se cachait là-dessous étaitsi fatiguée que le tissu était méconnaissable. Il eût fallu l'examinerde bien près et à la loupe pour reconnaître une moire ancienne démoirée,limée, coupée dans les plis, effrangée par en bas, et dévorée parla boue corrosive du pavé de Paris. Les souliers qui supportaient celamentable édifice n'avaient plus ni forme ni couleur. Le linge ne semontrait nulle part, ni au col, ni aux manches. Quelquefois, au passaged'un ruisseau, la robe se relevait à droite et laissait voir un bas delaine grise, un simple jupon de futaine noire. Les mains de la duchesse,rougies par un froid piquant, se cachaient sous son châle. Elle traînaitles pieds en marchant, non par une habitude de nonchalance, mais dans lapeur de perdre ses souliers. 172 | french#Par un contraste que vous avez pu observer quelquefois, la duchessen'était ni maigre, ni pâle, ni enlaidie en aucune façon par la misère. 173 | french#Elle avait reçu de ses ancêtres une de ces beautés rebelles quirésistent à tout, même à la faim. On a vu des prisonniers quiengraissaient dans leur cachot jusqu'à l'heure de la mort. A l'âge dequarante-sept ans, Mme de La Tour d'Embleuse conservait de beaux restesde jeunesse. Ses cheveux étaient noirs, et elle avait trente-deux dentscapables de broyer le pain le plus dur. Sa santé était moins florissanteque sa figure, mais c'est un secret qui restait entre elle et sonmédecin. La duchesse touchait à cette heure dangereuse et quelquefoismortelle où la femme disparaît pour faire place à l'aïeule. Plus d'unefois elle avait été saisie par des suffocations étranges. Elle rêvaitsouvent que le sang la prenait à la gorge pour l'étouffer. Des chaleursinexplicables lui montaient au cerveau par bouffées, et elle s'éveillaitdans un bain de vapeur animale où elle s'étonnait de ne point mourir. Ledocteur Le Bris, un jeune médecin et un vieil ami, lui recommandait unrégime doux, sans fatigues et surtout sans émotions. Mais quelle âmestoïcienne aurait traversé sans s'émouvoir de si rudes épreuves? 174 | french#Le duc César de La Tour d'Embleuse, fils d'un des émigrés les plusfidèles au roi et les plus acharnés contre le pays, fut récompensé 175 | french#magnifiquement des services de son père. En 1827, Charles X le nommagouverneur général de nos possessions dans l'Afrique occidentale. Ilétait à peine âgé de quarante ans. Pendant vingt-huit mois de séjourdans la colonie, il tint tête aux Maures et à la fièvre jaune; puis ildemanda un congé pour venir se marier à Paris. Il était riche, grâceau milliard d'indemnité; il doubla sa fortune en épousant la belleMarguerite de Bisson, qui possédait à Saint-Brieuc soixante mille livresde rente. Le roi signa son contrat le même jour que les ordonnances, etle duc se trouva marié et destitué tout d'un coup. Le nouveau pouvoirl'aurait accueilli volontiers dans la foule des transfuges; on dit mêmeque le ministère de Casimir Périer lui fit quelques avances. Il dédaignatous les emplois, par fierté d'abord, et autant par une invincibleparesse. Soit qu'il eût dépensé en trois ans tout ce qu'il avaitd'énergie, soit que la vie facile de Paris le retint par un attraitirrésistible, son seul travail pendant dix ans fut de promener seschevaux au Bois et de montrer ses gants jaunes au foyer de l'Opéra. 176 | english#The house stood on a slight rise just on the edge of the village. It stood on 177 | english#its own and looked over a broad spread of West Country farmland. Not a 178 | english#remarkable house by any means – it was about thirty years old, squattish, 179 | english#squarish, made of brick, and had four windows set in the front of a size and 180 | english#proportion which more or less exactly failed to please the eye. 181 | english#The only person for whom the house was in any way special was Arthur 182 | english#Dent, and that was only because it happened to be the one he lived in. He 183 | english#had lived in it for about three years, ever since he had moved out of London 184 | english#because it made him nervous and irritable. He was about thirty as well, 185 | english#dark haired and never quite at ease with himself. The thing that used to 186 | english#worry him most was the fact that people always used to ask him what he 187 | english#was looking so worried about. He worked in local radio which he always used 188 | english#to tell his friends was a lot more interesting than they probably thought. It 189 | english#was, too – most of his friends worked in advertising. 190 | english#On Wednesday night it had rained very heavily, the lane was wet and 191 | english#muddy, but the Thursday morning sun was bright and clear as it shone on 192 | english#Arthur Dent’s house for what was to be the last time 193 | english#It hadn’t properly registered with Arthur that the council wanted to 194 | english#knock down his house and build an bypass instead. 195 | english#At eight o’clock on Thursday morning Arthur didn’t feel very good. He 196 | english#woke up blearily, got up, wandered blearily round his room, opened a window, 197 | english#saw a bulldozer, found his slippers, and stomped off to the bathroom to wash. 198 | english#Toothpaste on the brush – so. Scrub. 199 | english#Shaving mirror – pointing at the ceiling. He adjusted it. For a moment 200 | english#it reflected a second bulldozer through the bathroom window. Properly adjusted, 201 | english#it reflected Arthur Dent’s bristles. He shaved them off, washed, dried, 202 | english#and stomped off to the kitchen to find something pleasant to put in his mouth. 203 | english#Kettle, plug, fridge, milk, coffee. Yawn. 204 | english#The word bulldozer wandered through his mind for a moment in search 205 | english#of something to connect with. 206 | english#The bulldozer outside the kitchen window was quite a big one. 207 | english#He stared at it. ”Yellow,” he thought and stomped off back to his bedroom 208 | english#to get dressed. 209 | english#Passing the bathroom he stopped to drink a large glass of water, and 210 | english#another. He began to suspect that he was hung over. Why was he hung 211 | english#over? Had he been drinking the night before? He supposed that he must 212 | english#have been. He caught a glint in the shaving mirror. ”Yellow,” he thought 213 | english#and stomped on to the bedroom. 214 | english#He stood and thought. The pub, he thought. Oh dear, the pub. He 215 | english#vaguely remembered being angry, angry about something that seemed important. 216 | english#He’d been telling people about it, telling people about it at great 217 | english#length, he rather suspected: his clearest visual recollection was of glazed 218 | english#looks on other people’s faces. Something about a new bypass he had just 219 | english#found out about. It had been in the pipeline for months only no one seemed 220 | english#to have known about it. Ridiculous. He took a swig of water. It would sort 221 | english#itself out, he’d decided, no one wanted a bypass, the council didn’t have a 222 | english#leg to stand on. It would sort itself out. 223 | english#God what a terrible hangover it had earned him though. He looked 224 | english#at himself in the wardrobe mirror. He stuck out his tongue. ”Yellow,” he 225 | english#thought. The word yellow wandered through his mind in search of something 226 | english#to connect with. 227 | english#Fifteen seconds later he was out of the house and lying in front of a big 228 | english#yellow bulldozer that was advancing up his garden path. 229 | english#Mr. L. Prosser was, as they say, only human. In other words he was a 230 | english#carbon-based life form descended from an ape. More specifically he was forty, 231 | english#fat and shabby and worked for the local council. Curiously enough, though 232 | english#he didn’t know it, he was also a direct male-line descendant of Genghis Khan, 233 | english#though intervening generations and racial mixing had so juggled his genes 234 | english#that he had no discernible Mongoloid characteristics, and the only vestiges 235 | english#left in Mr. L. Prosser of his mighty ancestry were a pronounced stoutness 236 | english#about the tum and a predilection for little fur hats. 237 | english#He was by no means a great warrior: in fact he was a nervous worried 238 | english#man. Today he was particularly nervous and worried because something had 239 | english#gone seriously wrong with his job – which was to see that Arthur Dent’s 240 | english#house got cleared out of the way before the day was out. 241 | english#”Come off it, Mr. Dent,”, he said, ”you can’t win you know. You can’t lie 242 | english#in front of the bulldozer indefinitely.” He tried to make his eyes blaze fiercely 243 | english#but they just wouldn’t do it. 244 | english#Arthur lay in the mud and squelched at him. 245 | --------------------------------------------------------------------------------